From 64af9bfe5bfcbe81e865c7c2bb3b898297b1d76c Mon Sep 17 00:00:00 2001 From: George Shiqi Wu Date: Fri, 16 Jun 2023 12:28:16 -0400 Subject: [PATCH 01/74] Add groupId to metrics (#14402) * Add group id as a dimension * Revert changes * Add to forking task runner * Add missing metrics * Fix indenting * revert metrics * Fix indentation --- docs/operations/metrics.md | 60 +++++++++---------- .../TaskRealtimeMetricsMonitorBuilder.java | 3 +- .../common/task/AbstractBatchIndexTask.java | 1 + .../indexing/common/task/IndexTaskUtils.java | 2 + .../indexing/overlord/ForkingTaskRunner.java | 4 ++ .../common/task/IndexTaskUtilsTest.java | 34 +++++++++++ .../org/apache/druid/query/DruidMetrics.java | 1 + 7 files changed, 74 insertions(+), 31 deletions(-) diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index cb786abb823..16aaca90837 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -154,9 +154,9 @@ If SQL is enabled, the Broker will emit the following metrics for SQL. |Metric|Description| Dimensions |Normal Value| |------|-----------|---------------------------------------------------------|------------| -|`ingest/count`|Count of `1` every time an ingestion job runs (includes compaction jobs). Aggregate using dimensions. | `dataSource`, `taskId`, `taskType`, `taskIngestionMode`, `tags` |Always `1`.| -|`ingest/segments/count`|Count of final segments created by job (includes tombstones). | `dataSource`, `taskId`, `taskType`, `taskIngestionMode`, `tags` |At least `1`.| -|`ingest/tombstones/count`|Count of tombstones created by job. | `dataSource`, `taskId`, `taskType`, `taskIngestionMode`, `tags` |Zero or more for replace. Always zero for non-replace tasks (always zero for legacy replace, see below).| +|`ingest/count`|Count of `1` every time an ingestion job runs (includes compaction jobs). Aggregate using dimensions. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskIngestionMode`, `tags` |Always `1`.| +|`ingest/segments/count`|Count of final segments created by job (includes tombstones). | `dataSource`, `taskId`, `taskType`, `groupId`, `taskIngestionMode`, `tags` |At least `1`.| +|`ingest/tombstones/count`|Count of tombstones created by job. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskIngestionMode`, `tags` |Zero or more for replace. Always zero for non-replace tasks (always zero for legacy replace, see below).| The `taskIngestionMode` dimension includes the following modes: * `APPEND`: a native ingestion job appending to existing segments @@ -206,26 +206,26 @@ batch ingestion emit the following metrics. These metrics are deltas for each em |Metric|Description|Dimensions|Normal Value| |------|-----------|----------|------------| -|`ingest/events/thrownAway`|Number of events rejected because they are either null, or filtered by the transform spec, or outside the windowPeriod.|`dataSource`, `taskId`, `taskType`, `tags`|0| -|`ingest/events/unparseable`|Number of events rejected because the events are unparseable.|`dataSource`, `taskId`, `taskType`, `tags`|0| -|`ingest/events/duplicate`|Number of events rejected because the events are duplicated.|`dataSource`, `taskId`, `taskType`, `tags`|0| -|`ingest/events/processed`|Number of events successfully processed per emission period.|`dataSource`, `taskId`, `taskType`, `tags`|Equal to the number of events per emission period.| -|`ingest/rows/output`|Number of Druid rows persisted.|`dataSource`, `taskId`, `taskType`|Your number of events with rollup.| -|`ingest/persists/count`|Number of times persist occurred.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration.| -|`ingest/persists/time`|Milliseconds spent doing intermediate persist.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration. Generally a few minutes at most.| -|`ingest/persists/cpu`|Cpu time in Nanoseconds spent on doing intermediate persist.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration. Generally a few minutes at most.| -|`ingest/persists/backPressure`|Milliseconds spent creating persist tasks and blocking waiting for them to finish.|`dataSource`, `taskId`, `taskType`, `tags`|0 or very low| -|`ingest/persists/failed`|Number of persists that failed.|`dataSource`, `taskId`, `taskType`, `tags`|0| -|`ingest/handoff/failed`|Number of handoffs that failed.|`dataSource`, `taskId`, `taskType`, `tags`|0| -|`ingest/merge/time`|Milliseconds spent merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration. Generally a few minutes at most.| -|`ingest/merge/cpu`|Cpu time in Nanoseconds spent on merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration. Generally a few minutes at most.| -|`ingest/handoff/count`|Number of handoffs that happened.|`dataSource`, `taskId`, `taskType`, `tags`|Varies. Generally greater than 0 once every segment granular period if cluster operating normally.| -|`ingest/sink/count`|Number of sinks not handoffed.|`dataSource`, `taskId`, `taskType`, `tags`|1~3| -|`ingest/events/messageGap`|Time gap in milliseconds between the latest ingested event timestamp and the current system timestamp of metrics emission. If the value is increasing but lag is low, Druid may not be receiving new data. This metric is reset as new tasks spawn up.|`dataSource`, `taskId`, `taskType`, `tags`|Greater than 0, depends on the time carried in event. | +|`ingest/events/thrownAway`|Number of events rejected because they are either null, or filtered by the transform spec, or outside the windowPeriod.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0| +|`ingest/events/unparseable`|Number of events rejected because the events are unparseable.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0| +|`ingest/events/duplicate`|Number of events rejected because the events are duplicated.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0| +|`ingest/events/processed`|Number of events successfully processed per emission period.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Equal to the number of events per emission period.| +|`ingest/rows/output`|Number of Druid rows persisted.|`dataSource`, `taskId`, `taskType`, `groupId`|Your number of events with rollup.| +|`ingest/persists/count`|Number of times persist occurred.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration.| +|`ingest/persists/time`|Milliseconds spent doing intermediate persist.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration. Generally a few minutes at most.| +|`ingest/persists/cpu`|Cpu time in Nanoseconds spent on doing intermediate persist.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration. Generally a few minutes at most.| +|`ingest/persists/backPressure`|Milliseconds spent creating persist tasks and blocking waiting for them to finish.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0 or very low| +|`ingest/persists/failed`|Number of persists that failed.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0| +|`ingest/handoff/failed`|Number of handoffs that failed.|`dataSource`, `taskId`, `taskType`, `groupId`,`tags`|0| +|`ingest/merge/time`|Milliseconds spent merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration. Generally a few minutes at most.| +|`ingest/merge/cpu`|Cpu time in Nanoseconds spent on merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration. Generally a few minutes at most.| +|`ingest/handoff/count`|Number of handoffs that happened.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Varies. Generally greater than 0 once every segment granular period if cluster operating normally.| +|`ingest/sink/count`|Number of sinks not handoffed.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|1~3| +|`ingest/events/messageGap`|Time gap in milliseconds between the latest ingested event timestamp and the current system timestamp of metrics emission. If the value is increasing but lag is low, Druid may not be receiving new data. This metric is reset as new tasks spawn up.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Greater than 0, depends on the time carried in event. | |`ingest/notices/queueSize`|Number of pending notices to be processed by the coordinator.|`dataSource`, `tags`|Typically 0 and occasionally in lower single digits. Should not be a very high number. | |`ingest/notices/time`|Milliseconds taken to process a notice by the supervisor.|`dataSource`, `tags`| < 1s | |`ingest/pause/time`|Milliseconds spent by a task in a paused state without ingesting.|`dataSource`, `taskId`, `tags`| < 10 seconds| -|`ingest/handoff/time`|Total number of milliseconds taken to handoff a set of segments.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on coordinator cycle time.| +|`ingest/handoff/time`|Total number of milliseconds taken to handoff a set of segments.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on coordinator cycle time.| Note: If the JVM does not support CPU time measurement for the current thread, `ingest/merge/cpu` and `ingest/persists/cpu` will be 0. @@ -233,20 +233,20 @@ Note: If the JVM does not support CPU time measurement for the current thread, ` |Metric|Description| Dimensions |Normal Value| |------|-----------|------------------------------------------------------------|------------| -|`task/run/time`|Milliseconds taken to run a task.| `dataSource`, `taskId`, `taskType`, `taskStatus`, `tags`|Varies| -|`task/pending/time`|Milliseconds taken for a task to wait for running.| `dataSource`, `taskId`, `taskType`, `tags`|Varies| -|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.| `dataSource`, `taskId`, `taskType`, `taskActionType`, `tags`|< 1000 (subsecond)| -|`task/action/run/time`|Milliseconds taken to execute a task action.| `dataSource`, `taskId`, `taskType`, `taskActionType`, `tags`|Varies from subsecond to a few seconds, based on action type.| -|`task/action/success/count`|Number of task actions that were executed successfully during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `taskActionType`, `tags`|Varies| -|`task/action/failed/count`|Number of task actions that failed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `taskActionType`, `tags`|Varies| +|`task/run/time`|Milliseconds taken to run a task.| `dataSource`, `taskId`, `taskType`, `groupId`, `taskStatus`, `tags`|Varies| +|`task/pending/time`|Milliseconds taken for a task to wait for running.| `dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Varies| +|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.| `dataSource`, `taskId`, `taskType`, `groupId`, `taskActionType`, `tags`|< 1000 (subsecond)| +|`task/action/run/time`|Milliseconds taken to execute a task action.| `dataSource`, `taskId`, `taskType`, `groupId`, `taskActionType`, `tags`|Varies from subsecond to a few seconds, based on action type.| +|`task/action/success/count`|Number of task actions that were executed successfully during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `groupId`, `taskActionType`, `tags`|Varies| +|`task/action/failed/count`|Number of task actions that failed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `groupId`, `taskActionType`, `tags`|Varies| |`task/action/batch/queueTime`|Milliseconds spent by a batch of task actions in queue. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies based on the `batchAllocationWaitTime` and number of batches in queue.| |`task/action/batch/runTime`|Milliseconds taken to execute a batch of task actions. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies from subsecond to a few seconds, based on action type and batch size.| |`task/action/batch/size`|Number of task actions in a batch that was executed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies based on number of concurrent task actions.| |`task/action/batch/attempts`|Number of execution attempts for a single batch of task actions. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|1 if there are no failures or retries.| -|`task/segmentAvailability/wait/time`|The amount of milliseconds a batch indexing task waited for newly created segments to become available for querying.| `dataSource`, `taskType`, `taskId`, `segmentAvailabilityConfirmed`, `tags`|Varies| -|`segment/added/bytes`|Size in bytes of new segments created.| `dataSource`, `taskId`, `taskType`, `interval`, `tags`|Varies| -|`segment/moved/bytes`|Size in bytes of segments moved/archived via the Move Task.| `dataSource`, `taskId`, `taskType`, `interval`, `tags`|Varies| -|`segment/nuked/bytes`|Size in bytes of segments deleted via the Kill Task.| `dataSource`, `taskId`, `taskType`, `interval`, `tags`|Varies| +|`task/segmentAvailability/wait/time`|The amount of milliseconds a batch indexing task waited for newly created segments to become available for querying.| `dataSource`, `taskType`, `groupId`, `taskId`, `segmentAvailabilityConfirmed`, `tags`|Varies| +|`segment/added/bytes`|Size in bytes of new segments created.| `dataSource`, `taskId`, `taskType`, `groupId`, `interval`, `tags`|Varies| +|`segment/moved/bytes`|Size in bytes of segments moved/archived via the Move Task.| `dataSource`, `taskId`, `taskType`, `groupId`, `interval`, `tags`|Varies| +|`segment/nuked/bytes`|Size in bytes of segments deleted via the Kill Task.| `dataSource`, `taskId`, `taskType`, `groupId`, `interval`, `tags`|Varies| |`task/success/count`|Number of successful tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.| `dataSource`|Varies| |`task/failed/count`|Number of failed tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.|`dataSource`|Varies| |`task/running/count`|Number of current running tasks. This metric is only available if the `TaskCountStatsMonitor` module is included.|`dataSource`|Varies| diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorBuilder.java index a07ad4eaad9..abd95169073 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorBuilder.java @@ -56,7 +56,8 @@ public class TaskRealtimeMetricsMonitorBuilder meters, ImmutableMap.of( DruidMetrics.TASK_ID, new String[]{task.getId()}, - DruidMetrics.TASK_TYPE, new String[]{task.getType()} + DruidMetrics.TASK_TYPE, new String[]{task.getType()}, + DruidMetrics.GROUP_ID, new String[]{task.getGroupId()} ), task.getContextValue(DruidMetrics.TAGS) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index a4cd183c01b..11bcfb6de78 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -650,6 +650,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask .setDimension("dataSource", getDataSource()) .setDimension("taskType", getType()) .setDimension("taskId", getId()) + .setDimension("groupId", getGroupId()) .setDimensionIfNotNull(DruidMetrics.TAGS, getContextValue(DruidMetrics.TAGS)) .setDimension("segmentAvailabilityConfirmed", segmentAvailabilityConfirmationCompleted) .build("task/segmentAvailability/wait/time", segmentAvailabilityWaitTimeMs) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java index 05be8c19410..cd7a52f7728 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java @@ -117,6 +117,7 @@ public class IndexTaskUtils DruidMetrics.TAGS, task.>getContextValue(DruidMetrics.TAGS) ); + metricBuilder.setDimensionIfNotNull(DruidMetrics.GROUP_ID, task.getGroupId()); } public static void setTaskDimensions(final ServiceMetricEvent.Builder metricBuilder, final AbstractTask task) @@ -129,6 +130,7 @@ public class IndexTaskUtils DruidMetrics.TAGS, task.>getContextValue(DruidMetrics.TAGS) ); + metricBuilder.setDimensionIfNotNull(DruidMetrics.GROUP_ID, task.getGroupId()); } public static void setTaskStatusDimensions( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java index 8df3a69eee2..c2f2c0363ed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java @@ -314,6 +314,10 @@ public class ForkingTaskRunner MonitorsConfig.METRIC_DIMENSION_PREFIX + DruidMetrics.TASK_TYPE, task.getType() ); + command.addSystemProperty( + MonitorsConfig.METRIC_DIMENSION_PREFIX + DruidMetrics.GROUP_ID, + task.getGroupId() + ); command.addSystemProperty("druid.host", childHost); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskUtilsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskUtilsTest.java index 8543f893fd9..025b3ee09ea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskUtilsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskUtilsTest.java @@ -36,6 +36,8 @@ import java.util.Map; public class IndexTaskUtilsTest { private static final Map METRIC_TAGS = ImmutableMap.of("k1", "v1", "k2", 20); + + private static final String GROUP_ID = "groupId123"; @Mock private Task task; @Mock @@ -47,7 +49,9 @@ public class IndexTaskUtilsTest { metricBuilder = ServiceMetricEvent.builder(); Mockito.when(task.getContextValue(DruidMetrics.TAGS)).thenReturn(METRIC_TAGS); + Mockito.when(task.getGroupId()).thenReturn(GROUP_ID); Mockito.when(abstractTask.getContextValue(DruidMetrics.TAGS)).thenReturn(METRIC_TAGS); + Mockito.when(abstractTask.getGroupId()).thenReturn(GROUP_ID); } @Test @@ -79,4 +83,34 @@ public class IndexTaskUtilsTest IndexTaskUtils.setTaskDimensions(metricBuilder, abstractTask); Assert.assertNull(metricBuilder.getDimension(DruidMetrics.TAGS)); } + + @Test + public void testSetTaskDimensionsWithGroupIdShouldSetGroupId() + { + IndexTaskUtils.setTaskDimensions(metricBuilder, task); + Assert.assertEquals(GROUP_ID, metricBuilder.getDimension(DruidMetrics.GROUP_ID)); + } + + @Test + public void testSetTaskDimensionsWithoutGroupIdShouldNotSetGroupId() + { + Mockito.when(task.getGroupId()).thenReturn(null); + IndexTaskUtils.setTaskDimensions(metricBuilder, task); + Assert.assertNull(metricBuilder.getDimension(DruidMetrics.GROUP_ID)); + } + + @Test + public void testSetTaskDimensionsForAbstractTaskWithGroupIdShouldSetGroupId() + { + IndexTaskUtils.setTaskDimensions(metricBuilder, abstractTask); + Assert.assertEquals(GROUP_ID, metricBuilder.getDimension(DruidMetrics.GROUP_ID)); + } + + @Test + public void testSetTaskDimensionsForAbstractTaskWithoutGroupIdShouldNotSetGroupId() + { + Mockito.when(abstractTask.getGroupId()).thenReturn(null); + IndexTaskUtils.setTaskDimensions(metricBuilder, abstractTask); + Assert.assertNull(metricBuilder.getDimension(DruidMetrics.GROUP_ID)); + } } diff --git a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java index 50482ce7f44..5caf90d3fda 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java @@ -33,6 +33,7 @@ public class DruidMetrics public static final String INTERVAL = "interval"; public static final String ID = "id"; public static final String TASK_ID = "taskId"; + public static final String GROUP_ID = "groupId"; public static final String STATUS = "status"; public static final String TASK_INGESTION_MODE = "taskIngestionMode"; From 04fb75719e7dd76d051dbd4cf8f9c2712b227793 Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Fri, 16 Jun 2023 15:10:12 -0700 Subject: [PATCH 02/74] Fail query planning if a `CLUSTERED BY` column contains descending order (#14436) * Throw ValidationException if CLUSTERED BY column descending order is specified. - Fails query planning * Some more tests. * fixup existing comment * Update comment * checkstyle fix: remove unused imports * Remove InsertCannotOrderByDescendingFault and deprecate the fault in readme. * move deprecated field to the bottom --- docs/multi-stage-query/reference.md | 2 +- .../apache/druid/msq/exec/ControllerImpl.java | 5 -- .../druid/msq/guice/MSQIndexingModule.java | 2 - .../InsertCannotOrderByDescendingFault.java | 72 ------------------ .../apache/druid/msq/exec/MSQFaultsTest.java | 18 ----- .../apache/druid/msq/exec/MSQInsertTest.java | 19 +++++ .../apache/druid/msq/exec/MSQReplaceTest.java | 21 +++++- .../msq/indexing/error/MSQFaultSerdeTest.java | 1 - .../calcite/parser/DruidSqlParserUtils.java | 33 ++++++++- .../sql/calcite/CalciteInsertDmlTest.java | 38 +++++++--- .../sql/calcite/CalciteReplaceDmlTest.java | 20 +++++ .../parser/DruidSqlParserUtilsTest.java | 73 ++++++++++++++++++- .../insertWithClusteredBy-logicalPlan.txt | 4 +- 13 files changed, 192 insertions(+), 116 deletions(-) delete mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotOrderByDescendingFault.java diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index b28b3a2f5be..3e54abfce8b 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -425,7 +425,6 @@ The following table describes error codes you may encounter in the `multiStageQu | `ColumnTypeNotSupported` | The column type is not supported. This can be because:

  • Support for writing or reading from a particular column type is not supported.
  • The query attempted to use a column type that is not supported by the frame format. This occurs with ARRAY types, which are not yet implemented for frames.
| `columnName`: The column name with an unsupported type.

`columnType`: The unknown column type. | | `InsertCannotAllocateSegment` | The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts:

  • Attempting to mix different granularities in the same intervals of the same datasource.
  • Prior ingestions that used non-extendable shard specs.
| `dataSource`

`interval`: The interval for the attempted new segment allocation. | | `InsertCannotBeEmpty` | An INSERT or REPLACE query did not generate any output rows in a situation where output rows are required for success. This can happen for INSERT or REPLACE queries with `PARTITIONED BY` set to something other than `ALL` or `ALL TIME`. | `dataSource` | -| `InsertCannotOrderByDescending` | An INSERT query contained a `CLUSTERED BY` expression in descending order. Druid's segment generation code only supports ascending order. | `columnName` | | `InsertLockPreempted` | An INSERT or REPLACE query was canceled by a higher-priority ingestion job, such as a real-time ingestion task. | | | `InsertTimeNull` | An INSERT or REPLACE query encountered a null timestamp in the `__time` field.

This can happen due to using an expression like `TIME_PARSE(timestamp) AS __time` with a timestamp that cannot be parsed. ([`TIME_PARSE`](../querying/sql-scalar.md#date-and-time-functions) returns null when it cannot parse a timestamp.) In this case, try parsing your timestamps using a different function or pattern. Or, if your timestamps may genuinely be null, consider using [`COALESCE`](../querying/sql-scalar.md#other-scalar-functions) to provide a default value. One option is [`CURRENT_TIMESTAMP`](../querying/sql-scalar.md#date-and-time-functions), which represents the start time of the job.| | `InsertTimeOutOfBounds` | A REPLACE query generated a timestamp outside the bounds of the TIMESTAMP parameter for your OVERWRITE WHERE clause.

To avoid this error, verify that the you specified is valid. | `interval`: time chunk interval corresponding to the out-of-bounds timestamp | @@ -449,3 +448,4 @@ The following table describes error codes you may encounter in the `multiStageQu | `WorkerFailed` | A worker task failed unexpectedly. | `errorMsg`

`workerTaskId`: The ID of the worker task. | | `WorkerRpcFailed` | A remote procedure call to a worker task failed and could not recover. | `workerTaskId`: the id of the worker task | | `UnknownError` | All other errors. | `message` | +| `InsertCannotOrderByDescending` | Deprecated. An INSERT query contained a `CLUSTERED BY` expression in descending order. Druid's segment generation code only supports ascending order. The query returns a `ValidationException` instead of the fault. | `columnName` | diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index a5ff643e8bf..dd163d406d3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -101,7 +101,6 @@ import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; import org.apache.druid.msq.indexing.error.FaultsExceededChecker; import org.apache.druid.msq.indexing.error.InsertCannotAllocateSegmentFault; import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; -import org.apache.druid.msq.indexing.error.InsertCannotOrderByDescendingFault; import org.apache.druid.msq.indexing.error.InsertLockPreemptedFault; import org.apache.druid.msq.indexing.error.InsertTimeOutOfBoundsFault; import org.apache.druid.msq.indexing.error.MSQErrorReport; @@ -1854,10 +1853,6 @@ public class ControllerImpl implements Controller // Such fields in CLUSTERED BY still control partitioning as expected, but do not affect sort order of rows // within an individual segment. for (final KeyColumn clusterByColumn : queryClusterBy.getColumns()) { - if (clusterByColumn.order() == KeyOrder.DESCENDING) { - throw new MSQException(new InsertCannotOrderByDescendingFault(clusterByColumn.columnName())); - } - final IntList outputColumns = columnMappings.getOutputColumnsForQueryColumn(clusterByColumn.columnName()); for (final int outputColumn : outputColumns) { outputColumnsInOrder.add(columnMappings.getOutputColumnName(outputColumn)); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 8cc7ab35bde..59300316d58 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -45,7 +45,6 @@ import org.apache.druid.msq.indexing.error.ColumnTypeNotSupportedFault; import org.apache.druid.msq.indexing.error.DurableStorageConfigurationFault; import org.apache.druid.msq.indexing.error.InsertCannotAllocateSegmentFault; import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; -import org.apache.druid.msq.indexing.error.InsertCannotOrderByDescendingFault; import org.apache.druid.msq.indexing.error.InsertLockPreemptedFault; import org.apache.druid.msq.indexing.error.InsertTimeNullFault; import org.apache.druid.msq.indexing.error.InsertTimeOutOfBoundsFault; @@ -113,7 +112,6 @@ public class MSQIndexingModule implements DruidModule DurableStorageConfigurationFault.class, InsertCannotAllocateSegmentFault.class, InsertCannotBeEmptyFault.class, - InsertCannotOrderByDescendingFault.class, InsertLockPreemptedFault.class, InsertTimeNullFault.class, InsertTimeOutOfBoundsFault.class, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotOrderByDescendingFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotOrderByDescendingFault.java deleted file mode 100644 index 43b50e87827..00000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotOrderByDescendingFault.java +++ /dev/null @@ -1,72 +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.druid.msq.indexing.error; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.common.base.Preconditions; - -import java.util.Objects; - -@JsonTypeName(InsertCannotOrderByDescendingFault.CODE) -public class InsertCannotOrderByDescendingFault extends BaseMSQFault -{ - static final String CODE = "InsertCannotOrderByDescending"; - - private final String columnName; - - @JsonCreator - public InsertCannotOrderByDescendingFault( - @JsonProperty("columnName") final String columnName - ) - { - super(CODE, "Cannot ingest column [%s] in descending order", columnName); - this.columnName = Preconditions.checkNotNull(columnName, "columnName"); - } - - @JsonProperty - public String getColumnName() - { - return columnName; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - InsertCannotOrderByDescendingFault that = (InsertCannotOrderByDescendingFault) o; - return Objects.equals(columnName, that.columnName); - } - - @Override - public int hashCode() - { - return Objects.hash(super.hashCode(), columnName); - } -} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java index c1da6ebc8e7..646286acaf5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.indexing.error.InsertCannotAllocateSegmentFault; import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; -import org.apache.druid.msq.indexing.error.InsertCannotOrderByDescendingFault; import org.apache.druid.msq.indexing.error.InsertTimeNullFault; import org.apache.druid.msq.indexing.error.InsertTimeOutOfBoundsFault; import org.apache.druid.msq.indexing.error.TooManyClusteredByColumnsFault; @@ -92,23 +91,6 @@ public class MSQFaultsTest extends MSQTestBase .verifyResults(); } - @Test - public void testInsertCannotOrderByDescendingFault() - { - RowSignature rowSignature = RowSignature.builder() - .add("__time", ColumnType.LONG) - .add("dim1", ColumnType.STRING) - .add("cnt", ColumnType.LONG).build(); - - // Add an DESC clustered by column, which should not be allowed - testIngestQuery().setSql( - "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null and __time < TIMESTAMP '2000-01-02 00:00:00' group by 1, 2 PARTITIONED by day clustered by dim1 DESC") - .setExpectedDataSource("foo1") - .setExpectedRowSignature(rowSignature) - .setExpectedMSQFault(new InsertCannotOrderByDescendingFault("d1")) - .verifyResults(); - } - @Test public void testInsertTimeOutOfBoundsFault() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index 548e5e01664..b55de6c165c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -736,6 +736,25 @@ public class MSQInsertTest extends MSQTestBase } + @Test + public void testInsertWithClusteredByDescendingThrowsException() + { + // Add a DESC clustered by column, which should not be allowed + testIngestQuery().setSql("INSERT INTO foo1 " + + "SELECT __time, dim1 , count(*) as cnt " + + "FROM foo " + + "GROUP BY 1, 2" + + "PARTITIONED BY DAY " + + "CLUSTERED BY dim1 DESC" + ) + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( + "[`dim1` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order.")) + )) + .verifyPlanningErrors(); + } + @Test public void testRollUpOnFoo1WithTimeFunctionComplexCol() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index eb1c459e66b..1dfd7742146 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -678,7 +678,7 @@ public class MSQReplaceTest extends MSQTestBase } @Test - public void testInsertOnFoo1Range() + public void testReplaceOnFoo1Range() { RowSignature rowSignature = RowSignature.builder() .add("__time", ColumnType.LONG) @@ -731,6 +731,25 @@ public class MSQReplaceTest extends MSQTestBase .verifyResults(); } + @Test + public void testReplaceWithClusteredByDescendingThrowsException() + { + // Add a DESC clustered by column, which should not be allowed + testIngestQuery().setSql(" REPLACE INTO foobar " + + "OVERWRITE ALL " + + "SELECT __time, m1, m2 " + + "FROM foo " + + "PARTITIONED BY ALL TIME " + + "CLUSTERED BY m2, m1 DESC" + ) + .setExpectedValidationErrorMatcher(CoreMatchers.allOf( + CoreMatchers.instanceOf(SqlPlanningException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( + "[`m1` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order.")) + )) + .verifyPlanningErrors(); + } + @Test public void testReplaceTombstonesOverPartiallyOverlappingSegments() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index 256397e9a23..484989e7bad 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -56,7 +56,6 @@ public class MSQFaultSerdeTest assertFaultSerde(new ColumnNameRestrictedFault("the column")); assertFaultSerde(new InsertCannotAllocateSegmentFault("the datasource", Intervals.ETERNITY)); assertFaultSerde(new InsertCannotBeEmptyFault("the datasource")); - assertFaultSerde(new InsertCannotOrderByDescendingFault("the column")); assertFaultSerde(InsertLockPreemptedFault.INSTANCE); assertFaultSerde(InsertTimeNullFault.INSTANCE); assertFaultSerde(new InsertTimeOutOfBoundsFault(Intervals.ETERNITY)); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java index cc67cff1199..9009237b780 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java @@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite.parser; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlIdentifier; @@ -256,9 +257,12 @@ public class DruidSqlParserUtils * @param query sql query * @param clusteredByList List of clustered by columns * @return SqlOrderBy node containing the clusteredByList information + * @throws ValidationException if any of the clustered by columns contain DESCENDING order. */ public static SqlOrderBy convertClusterByToOrderBy(SqlNode query, SqlNodeList clusteredByList) + throws ValidationException { + validateClusteredByColumns(clusteredByList); // If we have a CLUSTERED BY clause, extract the information in that CLUSTERED BY and create a new // SqlOrderBy node SqlNode offset = null; @@ -266,9 +270,9 @@ public class DruidSqlParserUtils if (query instanceof SqlOrderBy) { SqlOrderBy sqlOrderBy = (SqlOrderBy) query; - // This represents the underlying query free of OFFSET, FETCH and ORDER BY clauses - // For a sqlOrderBy.query like "SELECT dim1, sum(dim2) FROM foo OFFSET 10 FETCH 30 ORDER BY dim1 GROUP - // BY dim1 this would represent the "SELECT dim1, sum(dim2) from foo GROUP BY dim1 + // query represents the underlying query free of OFFSET, FETCH and ORDER BY clauses + // For a sqlOrderBy.query like "SELECT dim1, sum(dim2) FROM foo GROUP BY dim1 ORDER BY dim1 FETCH 30 OFFSET 10", + // this would represent the "SELECT dim1, sum(dim2) from foo GROUP BY dim1 query = sqlOrderBy.query; offset = sqlOrderBy.offset; fetch = sqlOrderBy.fetch; @@ -283,6 +287,29 @@ public class DruidSqlParserUtils ); } + /** + * Validates the clustered by columns to ensure that it does not contain DESCENDING order columns. + * + * @param clusteredByNodes List of SqlNodes representing columns to be clustered by. + * @throws ValidationException if any of the clustered by columns contain DESCENDING order. + */ + public static void validateClusteredByColumns(final SqlNodeList clusteredByNodes) throws ValidationException + { + if (clusteredByNodes == null) { + return; + } + + for (final SqlNode clusteredByNode : clusteredByNodes.getList()) { + if (clusteredByNode.isA(ImmutableSet.of(SqlKind.DESCENDING))) { + throw new ValidationException( + StringUtils.format("[%s] is invalid." + + " CLUSTERED BY columns cannot be sorted in descending order.", clusteredByNode.toString() + ) + ); + } + } + } + /** * This method is used to convert an {@link SqlNode} representing a query into a {@link DimFilter} for the same query. * It takes the timezone as a separate parameter, as Sql timestamps don't contain that information. Supported functions diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index 641b0ea49b4..aeabf5241a0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -653,11 +653,11 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest skipVectorize(); final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"druid.dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":\"2, `dim1` DESC, CEIL(`m2`)\"}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"druid.dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":\"2, `dim1`, CEIL(`m2`)\"}"; final String sql = "EXPLAIN PLAN FOR INSERT INTO druid.dst " + "SELECT __time, FLOOR(m1) as floor_m1, dim1, CEIL(m2) as ceil_m2 FROM foo " - + "PARTITIONED BY FLOOR(__time TO DAY) CLUSTERED BY 2, dim1 DESC, CEIL(m2)"; + + "PARTITIONED BY FLOOR(__time TO DAY) CLUSTERED BY 2, dim1, CEIL(m2)"; ObjectMapper queryJsonMapper = queryFramework().queryJsonMapper(); final ScanQuery expectedQuery = newScanQueryBuilder() @@ -671,7 +671,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .orderBy( ImmutableList.of( new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.DESCENDING), + new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING), new ScanQuery.OrderBy("v1", ScanQuery.Order.ASCENDING) ) ) @@ -718,7 +718,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest + "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"}," + "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}]," + "\"resultFormat\":\"compactedList\"," - + "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"descending\"}," + + "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"}," + "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"legacy\":false," + "\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"}," @@ -751,6 +751,24 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest didTest = true; } + @Test + public void testExplainPlanInsertWithClusteredByDescThrowsException() + { + skipVectorize(); + + final String sql = "EXPLAIN PLAN FOR INSERT INTO druid.dst " + + "SELECT __time, FLOOR(m1) as floor_m1, dim1, CEIL(m2) as ceil_m2 FROM foo " + + "PARTITIONED BY FLOOR(__time TO DAY) CLUSTERED BY 2, dim1 DESC, CEIL(m2)"; + + testIngestionQuery() + .sql(sql) + .expectValidationError( + SqlPlanningException.class, + "[`dim1` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order." + ) + .verify(); + } + @Test public void testInsertWithClusteredBy() { @@ -765,7 +783,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .sql( "INSERT INTO druid.dst " + "SELECT __time, FLOOR(m1) as floor_m1, dim1, CEIL(m2) as ceil_m2 FROM foo " - + "PARTITIONED BY FLOOR(__time TO DAY) CLUSTERED BY 2, dim1 DESC, CEIL(m2)" + + "PARTITIONED BY FLOOR(__time TO DAY) CLUSTERED BY 2, dim1, CEIL(m2)" ) .expectTarget("dst", targetRowSignature) .expectResources(dataSourceRead("foo"), dataSourceWrite("dst")) @@ -781,7 +799,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .orderBy( ImmutableList.of( new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.DESCENDING), + new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING), new ScanQuery.OrderBy("v1", ScanQuery.Order.ASCENDING) ) ) @@ -1052,7 +1070,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest final String query = "EXPLAIN PLAN FOR INSERT INTO druid.dst " + "SELECT __time, FLOOR(m1) as floor_m1, dim1, CEIL(m2) as ceil_m2 FROM foo " - + "PARTITIONED BY FLOOR(__time TO DAY) CLUSTERED BY 2, dim1 DESC, CEIL(m2)"; + + "PARTITIONED BY FLOOR(__time TO DAY) CLUSTERED BY 2, dim1 ASC, CEIL(m2)"; ObjectMapper queryJsonMapper = queryFramework().queryJsonMapper(); final ScanQuery expectedQuery = newScanQueryBuilder() @@ -1066,7 +1084,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .orderBy( ImmutableList.of( new ScanQuery.OrderBy("v0", ScanQuery.Order.ASCENDING), - new ScanQuery.OrderBy("dim1", ScanQuery.Order.DESCENDING), + new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING), new ScanQuery.OrderBy("v1", ScanQuery.Order.ASCENDING) ) ) @@ -1091,7 +1109,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest + "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"}," + "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}]," + "\"resultFormat\":\"compactedList\"," - + "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"descending\"}," + + "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"}," + "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"legacy\":false," + "\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"}," @@ -1101,7 +1119,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest + "}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"druid.dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":\"2, `dim1` DESC, CEIL(`m2`)\"}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"druid.dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":\"2, `dim1`, CEIL(`m2`)\"}"; // Use testQuery for EXPLAIN (not testIngestionQuery). testQuery( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index fb922ffe772..0c1f016600d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -765,6 +765,26 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest didTest = true; } + + @Test + public void testExplainPlanReplaceWithClusteredByDescThrowsException() + { + skipVectorize(); + + final String sql = "EXPLAIN PLAN FOR" + + " REPLACE INTO dst" + + " OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00' " + + "SELECT * FROM foo PARTITIONED BY DAY CLUSTERED BY dim1 DESC"; + + testIngestionQuery() + .sql(sql) + .expectValidationError( + SqlPlanningException.class, + "[`dim1` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order." + ) + .verify(); + } + @Test public void testExplainReplaceFromExternalUnauthorized() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java index 75c7f039941..1f295ea3587 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java @@ -21,13 +21,17 @@ package org.apache.druid.sql.calcite.parser; import com.google.common.collect.ImmutableList; import org.apache.calcite.avatica.util.TimeUnit; +import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlIntervalQualifier; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlPostfixOperator; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.tools.ValidationException; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.sql.calcite.expression.builtin.TimeFloorOperatorConversion; @@ -119,7 +123,74 @@ public class DruidSqlParserUtilsTest } } - public static class FloorToGranularityConversionTestErrors + public static class ClusteredByColumnsValidationTest + { + /** + * Tests an empty CLUSTERED BY clause + */ + @Test + public void testEmptyClusteredByColumnsValid() + { + final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); + try { + DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs); + } + catch (ValidationException e) { + Assert.fail("Did not expect an exception" + e.getMessage()); + } + } + + /** + * Tests clause "CLUSTERED BY DIM1, DIM2 ASC, 3" + */ + @Test + public void testClusteredByColumnsValid() + { + final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); + clusteredByArgs.add(new SqlIdentifier("DIM1", SqlParserPos.ZERO)); + clusteredByArgs.add(new SqlIdentifier("DIM2 ASC", SqlParserPos.ZERO)); + clusteredByArgs.add(SqlLiteral.createExactNumeric("3", SqlParserPos.ZERO)); + + try { + DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs); + } + catch (ValidationException e) { + Assert.fail("Did not expect an exception" + e.getMessage()); + } + } + + /** + * Tests clause "CLUSTERED BY DIM1, DIM2 ASC, 3, DIM4 DESC" + */ + @Test + public void testClusteredByColumnsWithDescThrowsException() + { + final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); + clusteredByArgs.add(new SqlIdentifier("DIM1", SqlParserPos.ZERO)); + clusteredByArgs.add(new SqlIdentifier("DIM2 ASC", SqlParserPos.ZERO)); + clusteredByArgs.add(SqlLiteral.createExactNumeric("3", SqlParserPos.ZERO)); + + final SqlBasicCall sqlBasicCall = new SqlBasicCall( + new SqlPostfixOperator("DESC", SqlKind.DESCENDING, 2, null, null, null), + new SqlNode[]{ + new SqlIdentifier("DIM4", SqlParserPos.ZERO) + }, + new SqlParserPos(0, 3) + ); + clusteredByArgs.add(sqlBasicCall); + + ValidationException e = Assert.assertThrows( + ValidationException.class, + () -> DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs) + ); + Assert.assertEquals( + "[`DIM4` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order.", + e.getMessage() + ); + } + } + + public static class FloorToGranularityConversionErrorsTest { /** * Tests clause like "PARTITIONED BY 'day'" diff --git a/sql/src/test/resources/calcite/expected/ingest/insertWithClusteredBy-logicalPlan.txt b/sql/src/test/resources/calcite/expected/ingest/insertWithClusteredBy-logicalPlan.txt index eb2d8501d5a..9eb25a81ec9 100644 --- a/sql/src/test/resources/calcite/expected/ingest/insertWithClusteredBy-logicalPlan.txt +++ b/sql/src/test/resources/calcite/expected/ingest/insertWithClusteredBy-logicalPlan.txt @@ -1,4 +1,4 @@ -LogicalInsert(target=[druid.dst], partitionedBy=[{type=period, period=P1D, timeZone=UTC, origin=null}], clusteredBy=[2, `dim1` DESC, CEIL(`m2`)]) - LogicalSort(sort0=[$1], sort1=[$2], sort2=[$3], dir0=[ASC], dir1=[DESC], dir2=[ASC]) +LogicalInsert(target=[druid.dst], partitionedBy=[{type=period, period=P1D, timeZone=UTC, origin=null}], clusteredBy=[2, `dim1`, CEIL(`m2`)]) + LogicalSort(sort0=[$1], sort1=[$2], sort2=[$3], dir0=[ASC], dir1=[ASC], dir2=[ASC]) LogicalProject(__time=[$0], floor_m1=[FLOOR($5)], dim1=[$1], ceil_m2=[CEIL($6)]) LogicalTableScan(table=[[druid, foo]]) From bd07c3dd43428608678e337c7935f237a64f533d Mon Sep 17 00:00:00 2001 From: George Shiqi Wu Date: Sat, 17 Jun 2023 20:30:37 -0400 Subject: [PATCH 03/74] Don't need to double synchronize on simple map operations (#14435) * Don't need to double syncronize on simple map operations * remove lock --- .../k8s/overlord/KubernetesTaskRunner.java | 62 ++++++++----------- 1 file changed, 25 insertions(+), 37 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index fe2f4be3711..30bea70416b 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -134,18 +134,16 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner @Override public ListenableFuture run(Task task) { - synchronized (tasks) { - tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> runTask(task)))); - return tasks.get(task.getId()).getResult(); - } + return tasks.computeIfAbsent( + task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> runTask(task))) + ).getResult(); } protected ListenableFuture joinAsync(Task task) { - synchronized (tasks) { - tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> joinTask(task)))); - return tasks.get(task.getId()).getResult(); - } + return tasks.computeIfAbsent( + task.getId(), k -> new KubernetesWorkItem(task, exec.submit(() -> joinTask(task))) + ).getResult(); } private TaskStatus runTask(Task task) @@ -163,20 +161,18 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner { KubernetesPeonLifecycle peonLifecycle = peonLifecycleFactory.build(task); - synchronized (tasks) { - KubernetesWorkItem workItem = tasks.get(task.getId()); + KubernetesWorkItem workItem = tasks.get(task.getId()); - if (workItem == null) { - throw new ISE("Task [%s] disappeared", task.getId()); - } - - if (workItem.isShutdownRequested()) { - throw new ISE("Task [%s] has been shut down", task.getId()); - } - - workItem.setKubernetesPeonLifecycle(peonLifecycle); + if (workItem == null) { + throw new ISE("Task [%s] disappeared", task.getId()); } + if (workItem.isShutdownRequested()) { + throw new ISE("Task [%s] has been shut down", task.getId()); + } + + workItem.setKubernetesPeonLifecycle(peonLifecycle); + try { TaskStatus taskStatus; if (run) { @@ -202,9 +198,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner } finally { - synchronized (tasks) { - tasks.remove(task.getId()); - } + tasks.remove(task.getId()); } } @@ -322,9 +316,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner @Override public Collection getKnownTasks() { - synchronized (tasks) { - return Lists.newArrayList(tasks.values()); - } + return Lists.newArrayList(tasks.values()); } @@ -393,23 +385,19 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner @Override public Collection getRunningTasks() { - synchronized (tasks) { - return tasks.values() - .stream() - .filter(KubernetesWorkItem::isRunning) - .collect(Collectors.toList()); - } + return tasks.values() + .stream() + .filter(KubernetesWorkItem::isRunning) + .collect(Collectors.toList()); } @Override public Collection getPendingTasks() { - synchronized (tasks) { - return tasks.values() - .stream() - .filter(KubernetesWorkItem::isPending) - .collect(Collectors.toList()); - } + return tasks.values() + .stream() + .filter(KubernetesWorkItem::isPending) + .collect(Collectors.toList()); } @Nullable From 128133fadcae97f9c45f9e529c229bef2cc96614 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 18 Jun 2023 10:02:21 +0530 Subject: [PATCH 04/74] Add column replication_factor column to sys.segments table (#14403) Description: Druid allows a configuration of load rules that may cause a used segment to not be loaded on any historical. This status is not tracked in the sys.segments table on the broker, which makes it difficult to determine if the unavailability of a segment is expected and if we should not wait for it to be loaded on a server after ingestion has finished. Changes: - Track replication factor in `SegmentReplicantLookup` during evaluation of load rules - Update API `/druid/coordinator/v1metadata/segments` to return replication factor - Add column `replication_factor` to the sys.segments virtual table and populate it in `MetadataSegmentView` - If this column is 0, the segment is not assigned to any historical and will not be loaded. --- docs/api-reference/api-reference.md | 4 +- docs/querying/sql-metadata-tables.md | 1 + .../auth_test_sys_schema_segments.json | 3 +- ...tatus.java => SegmentStatusInCluster.java} | 68 ++++--- ...t.java => SegmentStatusInClusterTest.java} | 54 ++++-- .../server/coordinator/DruidCoordinator.java | 20 ++ .../coordinator/SegmentReplicantLookup.java | 15 ++ .../server/coordinator/rules/LoadRule.java | 7 + .../druid/server/http/MetadataResource.java | 35 +++- .../server/http/MetadataResourceTest.java | 172 ++++++++++++++++++ .../calcite/schema/MetadataSegmentView.java | 40 ++-- .../sql/calcite/schema/SystemSchema.java | 27 ++- .../sql/calcite/schema/SystemSchemaTest.java | 44 +++-- website/.spelling | 1 + 14 files changed, 398 insertions(+), 93 deletions(-) rename processing/src/main/java/org/apache/druid/timeline/{SegmentWithOvershadowedStatus.java => SegmentStatusInCluster.java} (52%) rename processing/src/test/java/org/apache/druid/timeline/{SegmentWithOvershadowedStatusTest.java => SegmentStatusInClusterTest.java} (79%) create mode 100644 server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java diff --git a/docs/api-reference/api-reference.md b/docs/api-reference/api-reference.md index 9b762c08183..2a6c36deae1 100644 --- a/docs/api-reference/api-reference.md +++ b/docs/api-reference/api-reference.md @@ -173,11 +173,11 @@ Returns a list of all segments for one or more specific datasources enabled in t `GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus` -Returns a list of all segments for each datasource with the full segment metadata and an extra field `overshadowed`. +Returns a list of all segments for each datasource with the full segment metadata and extra fields `overshadowed` and `replicationFactor`. `GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&datasources={dataSourceName1}&datasources={dataSourceName2}` -Returns a list of all segments for one or more specific datasources with the full segment metadata and an extra field `overshadowed`. +Returns a list of all segments for one or more specific datasources with the full segment metadata and extra fields `overshadowed` and `replicationFactor`. `GET /druid/coordinator/v1/metadata/datasources` diff --git a/docs/querying/sql-metadata-tables.md b/docs/querying/sql-metadata-tables.md index 51ee58ef7a1..9b0a42b6c1d 100644 --- a/docs/querying/sql-metadata-tables.md +++ b/docs/querying/sql-metadata-tables.md @@ -157,6 +157,7 @@ Segments table provides details on all Druid segments, whether they are publishe |dimensions|VARCHAR|JSON-serialized form of the segment dimensions| |metrics|VARCHAR|JSON-serialized form of the segment metrics| |last_compaction_state|VARCHAR|JSON-serialized form of the compaction task's config (compaction task which created this segment). May be null if segment was not created by compaction task.| +|replication_factor|BIGINT|Total number of replicas of the segment that are required to be loaded across all historical tiers, based on the load rule that currently applies to this segment. If this value is 0, the segment is not assigned to any historical and will not be loaded. This value is -1 if load rules for the segment have not been evaluated yet.| For example, to retrieve all currently active segments for datasource "wikipedia", use the query: diff --git a/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json b/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json index 1cb97572db5..1ce7b44bc61 100644 --- a/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json +++ b/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json @@ -17,6 +17,7 @@ "shard_spec": "{\"type\":\"none\"}", "dimensions": "[\"anonymous\",\"area_code\",\"city\",\"continent_code\",\"country_name\",\"dma_code\",\"geo\",\"language\",\"namespace\",\"network\",\"newpage\",\"page\",\"postal_code\",\"region_lookup\",\"robot\",\"unpatrolled\",\"user\"]", "metrics": "[\"added\",\"count\",\"deleted\",\"delta\",\"delta_hist\",\"unique_users\",\"variation\"]", - "last_compaction_state": null + "last_compaction_state": null, + "replication_factor": 2 } ] diff --git a/processing/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java b/processing/src/main/java/org/apache/druid/timeline/SegmentStatusInCluster.java similarity index 52% rename from processing/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java rename to processing/src/main/java/org/apache/druid/timeline/SegmentStatusInCluster.java index 04c8bf4378b..4e5577f7603 100644 --- a/processing/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java +++ b/processing/src/main/java/org/apache/druid/timeline/SegmentStatusInCluster.java @@ -23,39 +23,55 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonUnwrapped; +import javax.annotation.Nullable; +import java.util.Objects; + /** - * DataSegment object plus the overshadowed status for the segment. An immutable object. - * - * SegmentWithOvershadowedStatus's {@link #compareTo} method considers only the {@link SegmentId} - * of the DataSegment object. + * This class represents the current state of a segment in the cluster and encapsulates the following: + *
    + *
  • the {@code DataSegment} object
  • + *
  • overshadowed status of the segment
  • + *
  • replication factor of the segment
  • + *
+ *

+ * Objects of this class are used to sync the state of segments from the Coordinator to different services, typically the Broker. + * The {@link #compareTo} method considers only the {@link SegmentId}. */ -public class SegmentWithOvershadowedStatus implements Comparable +public class SegmentStatusInCluster implements Comparable { private final boolean overshadowed; + /** + * The replication factor for the segment added across all tiers. This value is null if the load rules for + * the segment have not been evaluated yet. + */ + private final Integer replicationFactor; /** * dataSegment is serialized "unwrapped", i.e. it's properties are included as properties of - * enclosing class. If in future, if {@code SegmentWithOvershadowedStatus} were to extend {@link DataSegment}, + * enclosing class. If in the future, if {@code SegmentStatusInCluster} were to extend {@link DataSegment}, * there will be no change in the serialized format. */ @JsonUnwrapped private final DataSegment dataSegment; @JsonCreator - public SegmentWithOvershadowedStatus( - @JsonProperty("overshadowed") boolean overshadowed + public SegmentStatusInCluster( + @JsonProperty("overshadowed") boolean overshadowed, + @JsonProperty("replicationFactor") @Nullable Integer replicationFactor ) { // Jackson will overwrite dataSegment if needed (even though the field is final) - this(null, overshadowed); + this(null, overshadowed, replicationFactor); } - public SegmentWithOvershadowedStatus( + public SegmentStatusInCluster( DataSegment dataSegment, - boolean overshadowed + boolean overshadowed, + Integer replicationFactor ) { this.dataSegment = dataSegment; this.overshadowed = overshadowed; + this.replicationFactor = replicationFactor; } @JsonProperty @@ -70,35 +86,36 @@ public class SegmentWithOvershadowedStatus implements Comparable LOAD_SPEC = ImmutableMap.of("something", "or_other"); private static final boolean OVERSHADOWED = true; + private static final Integer REPLICATION_FACTOR = 2; private static final int TEST_VERSION = 0x9; - private static final SegmentWithOvershadowedStatus SEGMENT = createSegmentWithOvershadowedStatus(); + private static final SegmentStatusInCluster SEGMENT = createSegmentForTest(); private static ObjectMapper createObjectMapper() { @@ -59,7 +61,7 @@ public class SegmentWithOvershadowedStatusTest return objectMapper; } - private static SegmentWithOvershadowedStatus createSegmentWithOvershadowedStatus() + private static SegmentStatusInCluster createSegmentForTest() { DataSegment dataSegment = new DataSegment( "something", @@ -74,7 +76,7 @@ public class SegmentWithOvershadowedStatusTest 1 ); - return new SegmentWithOvershadowedStatus(dataSegment, OVERSHADOWED); + return new SegmentStatusInCluster(dataSegment, OVERSHADOWED, REPLICATION_FACTOR); } @Test @@ -85,7 +87,7 @@ public class SegmentWithOvershadowedStatusTest JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - Assert.assertEquals(11, objectMap.size()); + Assert.assertEquals(12, objectMap.size()); Assert.assertEquals("something", objectMap.get("dataSource")); Assert.assertEquals(INTERVAL.toString(), objectMap.get("interval")); Assert.assertEquals("1", objectMap.get("version")); @@ -96,12 +98,13 @@ public class SegmentWithOvershadowedStatusTest Assert.assertEquals(TEST_VERSION, objectMap.get("binaryVersion")); Assert.assertEquals(1, objectMap.get("size")); Assert.assertEquals(OVERSHADOWED, objectMap.get("overshadowed")); + Assert.assertEquals(REPLICATION_FACTOR, objectMap.get("replicationFactor")); final String json = MAPPER.writeValueAsString(SEGMENT); - final TestSegmentWithOvershadowedStatus deserializedSegment = MAPPER.readValue( + final TestSegment deserializedSegment = MAPPER.readValue( json, - TestSegmentWithOvershadowedStatus.class + TestSegment.class ); DataSegment dataSegment = SEGMENT.getDataSegment(); @@ -114,30 +117,33 @@ public class SegmentWithOvershadowedStatusTest Assert.assertEquals(dataSegment.getShardSpec(), deserializedSegment.getShardSpec()); Assert.assertEquals(dataSegment.getSize(), deserializedSegment.getSize()); Assert.assertEquals(dataSegment.getId(), deserializedSegment.getId()); + Assert.assertEquals(OVERSHADOWED, deserializedSegment.isOvershadowed()); + Assert.assertEquals(REPLICATION_FACTOR, deserializedSegment.getReplicationFactor()); } - // Previously, the implementation of SegmentWithOvershadowedStatus had @JsonCreator/@JsonProperty and @JsonUnwrapped + // Previously, the implementation of SegmentStatusInCluster had @JsonCreator/@JsonProperty and @JsonUnwrapped // on the same field (dataSegment), which used to work in Jackson 2.6, but does not work with Jackson 2.9: // https://github.com/FasterXML/jackson-databind/issues/265#issuecomment-264344051 @Test public void testJsonCreatorAndJsonUnwrappedAnnotationsAreCompatible() throws Exception { String json = MAPPER.writeValueAsString(SEGMENT); - SegmentWithOvershadowedStatus segment = MAPPER.readValue(json, SegmentWithOvershadowedStatus.class); + SegmentStatusInCluster segment = MAPPER.readValue(json, SegmentStatusInCluster.class); Assert.assertEquals(SEGMENT, segment); Assert.assertEquals(json, MAPPER.writeValueAsString(segment)); } } /** - * Subclass of DataSegment with overshadowed status + * Flat subclass of DataSegment for testing */ -class TestSegmentWithOvershadowedStatus extends DataSegment +class TestSegment extends DataSegment { private final boolean overshadowed; + private final Integer replicationFactor; @JsonCreator - public TestSegmentWithOvershadowedStatus( + public TestSegment( @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, @@ -154,7 +160,8 @@ class TestSegmentWithOvershadowedStatus extends DataSegment @JsonProperty("lasCompactionState") @Nullable CompactionState lastCompactionState, @JsonProperty("binaryVersion") Integer binaryVersion, @JsonProperty("size") long size, - @JsonProperty("overshadowed") boolean overshadowed + @JsonProperty("overshadowed") boolean overshadowed, + @JsonProperty("replicationFactor") Integer replicationFactor ) { super( @@ -170,6 +177,7 @@ class TestSegmentWithOvershadowedStatus extends DataSegment size ); this.overshadowed = overshadowed; + this.replicationFactor = replicationFactor; } @JsonProperty @@ -178,23 +186,31 @@ class TestSegmentWithOvershadowedStatus extends DataSegment return overshadowed; } + @JsonProperty + public Integer getReplicationFactor() + { + return replicationFactor; + } + @Override public boolean equals(Object o) { if (this == o) { return true; } - if (!(o instanceof TestSegmentWithOvershadowedStatus)) { + if (o == null || getClass() != o.getClass()) { return false; } if (!super.equals(o)) { return false; } - final TestSegmentWithOvershadowedStatus that = (TestSegmentWithOvershadowedStatus) o; - if (overshadowed != (that.overshadowed)) { - return false; - } - return true; + TestSegment that = (TestSegment) o; + return overshadowed == that.overshadowed && Objects.equals(replicationFactor, that.replicationFactor); } + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), overshadowed, replicationFactor); + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 8fce69e340b..e37ab05d6db 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -154,6 +154,13 @@ public class DruidCoordinator private volatile boolean started = false; private volatile SegmentReplicantLookup segmentReplicantLookup = null; + + /** + * Contains a map of segmentId to total replication factor across all tiers. This map is refreshed when load rules are + * evaluated. It is used by {@link DruidCoordinator} to supply this value to + * {@link org.apache.druid.server.http.MetadataResource}. + */ + private volatile Object2IntMap segmentIdToReplicationFactor = null; private volatile DruidCluster cluster = null; private int cachedBalancerThreadNumber; @@ -817,6 +824,12 @@ public class DruidCoordinator return ImmutableList.of(compactSegments); } + @Nullable + public Integer getReplicationFactorForSegment(SegmentId segmentId) + { + return segmentIdToReplicationFactor == null ? null : segmentIdToReplicationFactor.getInt(segmentId); + } + @VisibleForTesting protected class DutiesRunnable implements Runnable { @@ -943,6 +956,13 @@ public class DruidCoordinator } } } + + // Update the immutable replication factor map with latest values. + // This value is set here as it is recalculated during load rule evaluation. + if (params.getSegmentReplicantLookup() != null) { + segmentIdToReplicationFactor = params.getSegmentReplicantLookup().getSegmentIdToReplicationFactor(); + } + // Emit the runtime of the full DutiesRunnable params.getEmitter().emit( new ServiceMetricEvent.Builder() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java b/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java index 3a4bdb9f627..96e4fe81276 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java @@ -21,6 +21,8 @@ package org.apache.druid.server.coordinator; import com.google.common.collect.HashBasedTable; import com.google.common.collect.Table; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import it.unimi.dsi.fastutil.objects.Object2LongMap; import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.druid.client.ImmutableDruidServer; @@ -79,6 +81,7 @@ public class SegmentReplicantLookup private final Table segmentsInCluster; private final Table loadingSegments; + private final Map segmentIdToReplicationFactor = new HashMap<>(); private final DruidCluster cluster; private SegmentReplicantLookup( @@ -114,6 +117,18 @@ public class SegmentReplicantLookup return (retVal == null) ? 0 : retVal; } + // TODO: Refactor this setter, as this class is following a singleton pattern with only getters, and this breaks convention. + // This would be revamped in https://github.com/apache/druid/pull/13197 + public void setReplicationFactor(SegmentId segmentId, Integer requiredReplicas) + { + segmentIdToReplicationFactor.put(segmentId, requiredReplicas); + } + + public Object2IntMap getSegmentIdToReplicationFactor() + { + return new Object2IntOpenHashMap<>(segmentIdToReplicationFactor); + } + private int getLoadingReplicants(SegmentId segmentId, String tier) { Integer retVal = loadingSegments.get(segmentId, tier); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java index 933b63e4726..c89a74edbff 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java @@ -77,6 +77,8 @@ public abstract class LoadRule implements Rule targetReplicants.putAll(getTieredReplicants()); currentReplicants.putAll(params.getSegmentReplicantLookup().getClusterTiers(segment.getId())); + params.getSegmentReplicantLookup().setReplicationFactor(segment.getId(), getReplicationFactor()); + final CoordinatorStats stats = new CoordinatorStats(); assign(params, segment, stats); @@ -93,6 +95,11 @@ public abstract class LoadRule implements Rule } } + private int getReplicationFactor() + { + return getTieredReplicants().values().stream().reduce(0, Integer::sum); + } + @Override public boolean canLoadSegments() { diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index ade8ab992e4..0583cc87a16 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -32,13 +32,14 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.server.JettyUtils; +import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.http.security.DatasourceResourceFilter; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.SegmentWithOvershadowedStatus; +import org.apache.druid.timeline.SegmentStatusInCluster; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -69,18 +70,21 @@ public class MetadataResource private final SegmentsMetadataManager segmentsMetadataManager; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final AuthorizerMapper authorizerMapper; + private final DruidCoordinator coordinator; @Inject public MetadataResource( SegmentsMetadataManager segmentsMetadataManager, IndexerMetadataStorageCoordinator metadataStorageCoordinator, AuthorizerMapper authorizerMapper, + DruidCoordinator coordinator, @Json ObjectMapper jsonMapper ) { this.segmentsMetadataManager = segmentsMetadataManager; this.metadataStorageCoordinator = metadataStorageCoordinator; this.authorizerMapper = authorizerMapper; + this.coordinator = coordinator; } @GET @@ -140,7 +144,7 @@ public class MetadataResource ) { if (includeOvershadowedStatus != null) { - return getAllUsedSegmentsWithOvershadowedStatus(req, dataSources); + return getAllUsedSegmentsWithAdditionalDetails(req, dataSources); } Collection dataSourcesWithUsedSegments = @@ -165,7 +169,7 @@ public class MetadataResource return builder.entity(authorizedSegments).build(); } - private Response getAllUsedSegmentsWithOvershadowedStatus( + private Response getAllUsedSegmentsWithAdditionalDetails( HttpServletRequest req, @Nullable Set dataSources ) @@ -184,15 +188,30 @@ public class MetadataResource .flatMap(t -> t.getSegments().stream()); final Set overshadowedSegments = dataSourcesSnapshot.getOvershadowedSegments(); - final Stream usedSegmentsWithOvershadowedStatus = usedSegments - .map(segment -> new SegmentWithOvershadowedStatus(segment, overshadowedSegments.contains(segment))); + final Stream segmentStatus = usedSegments + .map(segment -> { + boolean isOvershadowed = overshadowedSegments.contains(segment); + Integer replicationFactor; + if (isOvershadowed) { + // If the segment is overshadowed, the replication factor won't be present in the coordinator, but we know + // that it should be 0 as we will unload it soon. + replicationFactor = 0; + } else { + replicationFactor = coordinator.getReplicationFactorForSegment(segment.getId()); + } + return new SegmentStatusInCluster( + segment, + isOvershadowed, + replicationFactor + ); + }); - final Function> raGenerator = segment -> Collections + final Function> raGenerator = segment -> Collections .singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSegment().getDataSource())); - final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources( + final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources( req, - usedSegmentsWithOvershadowedStatus::iterator, + segmentStatus::iterator, raGenerator, authorizerMapper ); diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java new file mode 100644 index 00000000000..f87b65d606d --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -0,0 +1,172 @@ +/* + * 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.druid.server.http; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.client.DataSourcesSnapshot; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthTestUtils; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentStatusInCluster; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.core.Response; +import java.util.ArrayList; +import java.util.List; + +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; + +public class MetadataResourceTest +{ + private static final String DATASOURCE1 = "datasource1"; + private static final String DATASOURCE2 = "datasource2"; + + private MetadataResource metadataResource; + + private SegmentsMetadataManager segmentsMetadataManager; + private DruidCoordinator coordinator; + private HttpServletRequest request; + + private final DataSegment dataSegment1 = new DataSegment( + DATASOURCE1, + Intervals.of("2010-01-01/P1D"), + "v0", + null, + null, + null, + null, + 0x9, + 10 + ); + + private final DataSegment dataSegment2 = new DataSegment( + DATASOURCE1, + Intervals.of("2010-01-22/P1D"), + "v0", + null, + null, + null, + null, + 0x9, + 20 + ); + + private final DataSegment dataSegment3 = new DataSegment( + DATASOURCE2, + Intervals.of("2010-01-01/P1M"), + "v0", + null, + null, + null, + null, + 0x9, + 30 + ); + + private final DataSegment dataSegment4 = new DataSegment( + DATASOURCE2, + Intervals.of("2010-01-02/P1D"), + "v0", + null, + null, + null, + null, + 0x9, + 35 + ); + + @Before + public void setUp() + { + request = mock(HttpServletRequest.class); + doReturn(mock(AuthenticationResult.class)).when(request).getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT); + + segmentsMetadataManager = mock(SegmentsMetadataManager.class); + ImmutableDruidDataSource druidDataSource1 = new ImmutableDruidDataSource( + DATASOURCE1, + ImmutableMap.of(), + ImmutableList.of( + dataSegment1, + dataSegment2 + ) + ); + + ImmutableDruidDataSource druidDataSource2 = new ImmutableDruidDataSource( + DATASOURCE1, + ImmutableMap.of(), + ImmutableList.of( + dataSegment3, + dataSegment4 + ) + ); + + DataSourcesSnapshot dataSourcesSnapshot = mock(DataSourcesSnapshot.class); + doReturn(dataSourcesSnapshot).when(segmentsMetadataManager).getSnapshotOfDataSourcesWithAllUsedSegments(); + doReturn(ImmutableList.of(druidDataSource1, druidDataSource2)).when(dataSourcesSnapshot).getDataSourcesWithAllUsedSegments(); + + coordinator = mock(DruidCoordinator.class); + doReturn(2).when(coordinator).getReplicationFactorForSegment(dataSegment1.getId()); + doReturn(null).when(coordinator).getReplicationFactorForSegment(dataSegment2.getId()); + doReturn(1).when(coordinator).getReplicationFactorForSegment(dataSegment3.getId()); + doReturn(1).when(coordinator).getReplicationFactorForSegment(dataSegment4.getId()); + doReturn(ImmutableSet.of(dataSegment4)).when(dataSourcesSnapshot).getOvershadowedSegments(); + + metadataResource = new MetadataResource(segmentsMetadataManager, mock(IndexerMetadataStorageCoordinator.class), AuthTestUtils.TEST_AUTHORIZER_MAPPER, coordinator, new ObjectMapper()); + } + + @Test + public void testGetAllSegmentsWithOvershadowedStatus() + { + Response response = metadataResource.getAllUsedSegments( + request, + null, + "includeOvershadowedStatus" + ); + + List resultList = materializeResponse(response); + Assert.assertEquals(resultList.size(), 4); + Assert.assertEquals(new SegmentStatusInCluster(dataSegment1, false, 2), resultList.get(0)); + Assert.assertEquals(new SegmentStatusInCluster(dataSegment2, false, null), resultList.get(1)); + Assert.assertEquals(new SegmentStatusInCluster(dataSegment3, false, 1), resultList.get(2)); + // Replication factor should be 0 as the segment is overshadowed + Assert.assertEquals(new SegmentStatusInCluster(dataSegment4, true, 0), resultList.get(3)); + } + + private List materializeResponse(Response response) + { + Iterable resultIterator = (Iterable) response.getEntity(); + List segmentStatusInClusters = new ArrayList<>(); + resultIterator.forEach(segmentStatusInClusters::add); + return segmentStatusInClusters; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java index 5d97ea394d2..c3a1ab48f5a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java @@ -22,6 +22,8 @@ package org.apache.druid.sql.calcite.schema; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; import com.google.common.collect.ImmutableSortedSet; import com.google.common.util.concurrent.Uninterruptibles; import com.google.inject.Inject; @@ -40,7 +42,8 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.sql.calcite.planner.SegmentMetadataCacheConfig; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentWithOvershadowedStatus; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.SegmentStatusInCluster; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import java.util.Iterator; @@ -76,7 +79,12 @@ public class MetadataSegmentView * from other threads. */ @MonotonicNonNull - private volatile ImmutableSortedSet publishedSegments = null; + private volatile ImmutableSortedSet publishedSegments = null; + /** + * Caches the replication factor for segment IDs. In case of coordinator restarts or leadership re-elections, the coordinator API returns `null` replication factor until load rules are evaluated. + * The cache can be used during these periods to continue serving the previously fetched values. + */ + private final Cache segmentIdToReplicationFactor; private final ScheduledExecutorService scheduledExec; private final long pollPeriodInMS; private final LifecycleLock lifecycleLock = new LifecycleLock(); @@ -97,6 +105,9 @@ public class MetadataSegmentView this.isCacheEnabled = config.isMetadataSegmentCacheEnable(); this.pollPeriodInMS = config.getMetadataSegmentPollPeriod(); this.scheduledExec = Execs.scheduledSingleThreaded("MetadataSegmentView-Cache--%d"); + this.segmentIdToReplicationFactor = CacheBuilder.newBuilder() + .expireAfterAccess(10, TimeUnit.MINUTES) + .build(); } @LifecycleStart @@ -133,27 +144,34 @@ public class MetadataSegmentView private void poll() { log.info("polling published segments from coordinator"); - final JsonParserIterator metadataSegments = getMetadataSegments( + final JsonParserIterator metadataSegments = getMetadataSegments( coordinatorDruidLeaderClient, jsonMapper, segmentWatcherConfig.getWatchedDataSources() ); - final ImmutableSortedSet.Builder builder = ImmutableSortedSet.naturalOrder(); + final ImmutableSortedSet.Builder builder = ImmutableSortedSet.naturalOrder(); while (metadataSegments.hasNext()) { - final SegmentWithOvershadowedStatus segment = metadataSegments.next(); + final SegmentStatusInCluster segment = metadataSegments.next(); final DataSegment interned = DataSegmentInterner.intern(segment.getDataSegment()); - final SegmentWithOvershadowedStatus segmentWithOvershadowedStatus = new SegmentWithOvershadowedStatus( + Integer replicationFactor = segment.getReplicationFactor(); + if (replicationFactor == null) { + replicationFactor = segmentIdToReplicationFactor.getIfPresent(segment.getDataSegment().getId()); + } else { + segmentIdToReplicationFactor.put(segment.getDataSegment().getId(), segment.getReplicationFactor()); + } + final SegmentStatusInCluster segmentStatusInCluster = new SegmentStatusInCluster( interned, - segment.isOvershadowed() + segment.isOvershadowed(), + replicationFactor ); - builder.add(segmentWithOvershadowedStatus); + builder.add(segmentStatusInCluster); } publishedSegments = builder.build(); cachePopulated.countDown(); } - Iterator getPublishedSegments() + Iterator getPublishedSegments() { if (isCacheEnabled) { Uninterruptibles.awaitUninterruptibly(cachePopulated); @@ -168,7 +186,7 @@ public class MetadataSegmentView } // Note that coordinator must be up to get segments - private JsonParserIterator getMetadataSegments( + private JsonParserIterator getMetadataSegments( DruidLeaderClient coordinatorClient, ObjectMapper jsonMapper, Set watchedDataSources @@ -188,7 +206,7 @@ public class MetadataSegmentView return SystemSchema.getThingsFromLeaderNode( query, - new TypeReference() + new TypeReference() { }, coordinatorClient, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 3e6cb3e58f2..2c60e9eda1c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -81,7 +81,7 @@ import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.table.RowSignatures; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.SegmentWithOvershadowedStatus; +import org.apache.druid.timeline.SegmentStatusInCluster; import org.jboss.netty.handler.codec.http.HttpMethod; import javax.annotation.Nullable; @@ -106,8 +106,8 @@ public class SystemSchema extends AbstractSchema private static final String TASKS_TABLE = "tasks"; private static final String SUPERVISOR_TABLE = "supervisors"; - private static final Function> - SEGMENT_WITH_OVERSHADOWED_STATUS_RA_GENERATOR = segment -> + private static final Function> + SEGMENT_STATUS_IN_CLUSTER_RA_GENERATOR = segment -> Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply( segment.getDataSegment().getDataSource()) ); @@ -117,6 +117,8 @@ public class SystemSchema extends AbstractSchema segment.getDataSource()) ); + private static final long REPLICATION_FACTOR_UNKNOWN = -1L; + /** * Booleans constants represented as long type, * where 1 = true and 0 = false to make it easy to count number of segments @@ -150,6 +152,7 @@ public class SystemSchema extends AbstractSchema .add("dimensions", ColumnType.STRING) .add("metrics", ColumnType.STRING) .add("last_compaction_state", ColumnType.STRING) + .add("replication_factor", ColumnType.LONG) .build(); static final RowSignature SERVERS_SIGNATURE = RowSignature @@ -288,7 +291,7 @@ public class SystemSchema extends AbstractSchema // Get published segments from metadata segment cache (if enabled in SQL planner config), else directly from // Coordinator. - final Iterator metadataStoreSegments = metadataView.getPublishedSegments(); + final Iterator metadataStoreSegments = metadataView.getPublishedSegments(); final Set segmentsAlreadySeen = Sets.newHashSetWithExpectedSize(druidSchema.cache().getTotalSegments()); @@ -326,7 +329,10 @@ public class SystemSchema extends AbstractSchema segment.getShardSpec() == null ? null : jsonMapper.writeValueAsString(segment.getShardSpec()), segment.getDimensions() == null ? null : jsonMapper.writeValueAsString(segment.getDimensions()), segment.getMetrics() == null ? null : jsonMapper.writeValueAsString(segment.getMetrics()), - segment.getLastCompactionState() == null ? null : jsonMapper.writeValueAsString(segment.getLastCompactionState()) + segment.getLastCompactionState() == null ? null : jsonMapper.writeValueAsString(segment.getLastCompactionState()), + // If the value is null, the load rules might have not evaluated yet, and we don't know the replication factor. + // This should be automatically updated in the next refesh with Coordinator. + val.getReplicationFactor() == null ? REPLICATION_FACTOR_UNKNOWN : (long) val.getReplicationFactor() }; } catch (JsonProcessingException e) { @@ -368,7 +374,8 @@ public class SystemSchema extends AbstractSchema val.getValue().getSegment().getShardSpec() == null ? null : jsonMapper.writeValueAsString(val.getValue().getSegment().getShardSpec()), val.getValue().getSegment().getDimensions() == null ? null : jsonMapper.writeValueAsString(val.getValue().getSegment().getDimensions()), val.getValue().getSegment().getMetrics() == null ? null : jsonMapper.writeValueAsString(val.getValue().getSegment().getMetrics()), - null // unpublished segments from realtime tasks will not be compacted yet + null, // unpublished segments from realtime tasks will not be compacted yet + REPLICATION_FACTOR_UNKNOWN // If the segment is unpublished, we won't have this information yet. }; } catch (JsonProcessingException e) { @@ -384,8 +391,8 @@ public class SystemSchema extends AbstractSchema } - private Iterator getAuthorizedPublishedSegments( - Iterator it, + private Iterator getAuthorizedPublishedSegments( + Iterator it, DataContext root ) { @@ -394,11 +401,11 @@ public class SystemSchema extends AbstractSchema "authenticationResult in dataContext" ); - final Iterable authorizedSegments = AuthorizationUtils + final Iterable authorizedSegments = AuthorizationUtils .filterAuthorizedResources( authenticationResult, () -> it, - SEGMENT_WITH_OVERSHADOWED_STATUS_RA_GENERATOR, + SEGMENT_STATUS_IN_CLUSTER_RA_GENERATOR, authorizerMapper ); return authorizedSegments.iterator(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 50d0f09a85e..511a76b18ad 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -99,7 +99,7 @@ import org.apache.druid.sql.calcite.util.TestServerInventoryView; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.SegmentWithOvershadowedStatus; +import org.apache.druid.timeline.SegmentStatusInCluster; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; import org.jboss.netty.handler.codec.http.DefaultHttpResponse; @@ -542,7 +542,7 @@ public class SystemSchemaTest extends CalciteTestBase final RelDataType rowType = segmentsTable.getRowType(new JavaTypeFactoryImpl()); final List fields = rowType.getFieldList(); - Assert.assertEquals(18, fields.size()); + Assert.assertEquals(19, fields.size()); final SystemSchema.TasksTable tasksTable = (SystemSchema.TasksTable) schema.getTableMap().get("tasks"); final RelDataType sysRowType = tasksTable.getRowType(new JavaTypeFactoryImpl()); @@ -564,12 +564,12 @@ public class SystemSchemaTest extends CalciteTestBase public void testSegmentsTable() throws Exception { final SegmentsTable segmentsTable = new SegmentsTable(druidSchema, metadataView, new ObjectMapper(), authMapper); - final Set publishedSegments = new HashSet<>(Arrays.asList( - new SegmentWithOvershadowedStatus(publishedCompactedSegment1, true), - new SegmentWithOvershadowedStatus(publishedCompactedSegment2, false), - new SegmentWithOvershadowedStatus(publishedUncompactedSegment3, false), - new SegmentWithOvershadowedStatus(segment1, true), - new SegmentWithOvershadowedStatus(segment2, false) + final Set publishedSegments = new HashSet<>(Arrays.asList( + new SegmentStatusInCluster(publishedCompactedSegment1, true, 2), + new SegmentStatusInCluster(publishedCompactedSegment2, false, 0), + new SegmentStatusInCluster(publishedUncompactedSegment3, false, 2), + new SegmentStatusInCluster(segment1, true, 2), + new SegmentStatusInCluster(segment2, false, 0) )); EasyMock.expect(metadataView.getPublishedSegments()).andReturn(publishedSegments.iterator()).once(); @@ -598,7 +598,8 @@ public class SystemSchemaTest extends CalciteTestBase 1L, //is_available 0L, //is_realtime 1L, //is_overshadowed - null //is_compacted + null, //is_compacted + 2L // replication_factor ); verifyRow( @@ -612,7 +613,8 @@ public class SystemSchemaTest extends CalciteTestBase 1L, //is_available 0L, //is_realtime 0L, //is_overshadowed, - null //is_compacted + null, //is_compacted + 0L // replication_factor ); //segment test3 is unpublished and has a NumberedShardSpec with partitionNum = 2 @@ -627,7 +629,8 @@ public class SystemSchemaTest extends CalciteTestBase 1L, //is_available 0L, //is_realtime 0L, //is_overshadowed - null //is_compacted + null, //is_compacted + -1L // replication_factor ); verifyRow( @@ -641,7 +644,8 @@ public class SystemSchemaTest extends CalciteTestBase 1L, //is_available 1L, //is_realtime 0L, //is_overshadowed - null //is_compacted + null, //is_compacted + -1L // replication_factor ); verifyRow( @@ -655,7 +659,8 @@ public class SystemSchemaTest extends CalciteTestBase 1L, //is_available 1L, //is_realtime 0L, //is_overshadowed - null //is_compacted + null, //is_compacted + -1L // replication_factor ); // wikipedia segments are published and unavailable, num_replicas is 0 @@ -671,7 +676,8 @@ public class SystemSchemaTest extends CalciteTestBase 0L, //is_available 0L, //is_realtime 1L, //is_overshadowed - expectedCompactionState //is_compacted + expectedCompactionState, //is_compacted + 2L // replication_factor ); verifyRow( @@ -685,7 +691,8 @@ public class SystemSchemaTest extends CalciteTestBase 0L, //is_available 0L, //is_realtime 0L, //is_overshadowed - expectedCompactionState //is_compacted + expectedCompactionState, //is_compacted + 0L // replication_factor ); verifyRow( @@ -699,7 +706,8 @@ public class SystemSchemaTest extends CalciteTestBase 0L, //is_available 0L, //is_realtime 0L, //is_overshadowed - null //is_compacted + null, //is_compacted + 2L // replication_factor ); // Verify value types. @@ -717,7 +725,8 @@ public class SystemSchemaTest extends CalciteTestBase long isAvailable, long isRealtime, long isOvershadowed, - CompactionState compactionState + CompactionState compactionState, + long replicationFactor ) throws Exception { Assert.assertEquals(segmentId, row[0].toString()); @@ -740,6 +749,7 @@ public class SystemSchemaTest extends CalciteTestBase } else { Assert.assertEquals(mapper.writeValueAsString(compactionState), row[17]); } + Assert.assertEquals(replicationFactor, row[18]); } @Test diff --git a/website/.spelling b/website/.spelling index 9a5a6c8352e..997d387c762 100644 --- a/website/.spelling +++ b/website/.spelling @@ -644,6 +644,7 @@ num_segments partition_num plaintext_port queue_insertion_time +replication_factor runner_status segment_id server_type From 2b676ac7f8e6bafd8369c4dc5c1d800a25c2e7ab Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sun, 18 Jun 2023 23:34:11 -0700 Subject: [PATCH 05/74] Quieter KafkaSupervisors in all bundled log4j2.xml. (#14444) Follow-up to #13392, which added this to a single log4j2.xml. --- examples/conf/druid/auto/_common/log4j2.xml | 5 +++++ examples/conf/druid/single-server/large/_common/log4j2.xml | 5 +++++ examples/conf/druid/single-server/medium/_common/log4j2.xml | 5 +++++ .../druid/single-server/micro-quickstart/_common/log4j2.xml | 5 +++++ .../druid/single-server/nano-quickstart/_common/log4j2.xml | 5 +++++ examples/conf/druid/single-server/small/_common/log4j2.xml | 5 +++++ examples/conf/druid/single-server/xlarge/_common/log4j2.xml | 5 +++++ 7 files changed, 35 insertions(+) diff --git a/examples/conf/druid/auto/_common/log4j2.xml b/examples/conf/druid/auto/_common/log4j2.xml index 0bda2f6968f..eb24a2a8940 100644 --- a/examples/conf/druid/auto/_common/log4j2.xml +++ b/examples/conf/druid/auto/_common/log4j2.xml @@ -82,5 +82,10 @@ + + + + + diff --git a/examples/conf/druid/single-server/large/_common/log4j2.xml b/examples/conf/druid/single-server/large/_common/log4j2.xml index 0bda2f6968f..eb24a2a8940 100644 --- a/examples/conf/druid/single-server/large/_common/log4j2.xml +++ b/examples/conf/druid/single-server/large/_common/log4j2.xml @@ -82,5 +82,10 @@ + + + + + diff --git a/examples/conf/druid/single-server/medium/_common/log4j2.xml b/examples/conf/druid/single-server/medium/_common/log4j2.xml index 0bda2f6968f..eb24a2a8940 100644 --- a/examples/conf/druid/single-server/medium/_common/log4j2.xml +++ b/examples/conf/druid/single-server/medium/_common/log4j2.xml @@ -82,5 +82,10 @@ + + + + + diff --git a/examples/conf/druid/single-server/micro-quickstart/_common/log4j2.xml b/examples/conf/druid/single-server/micro-quickstart/_common/log4j2.xml index 0bda2f6968f..eb24a2a8940 100644 --- a/examples/conf/druid/single-server/micro-quickstart/_common/log4j2.xml +++ b/examples/conf/druid/single-server/micro-quickstart/_common/log4j2.xml @@ -82,5 +82,10 @@ + + + + + diff --git a/examples/conf/druid/single-server/nano-quickstart/_common/log4j2.xml b/examples/conf/druid/single-server/nano-quickstart/_common/log4j2.xml index 0bda2f6968f..eb24a2a8940 100644 --- a/examples/conf/druid/single-server/nano-quickstart/_common/log4j2.xml +++ b/examples/conf/druid/single-server/nano-quickstart/_common/log4j2.xml @@ -82,5 +82,10 @@ + + + + + diff --git a/examples/conf/druid/single-server/small/_common/log4j2.xml b/examples/conf/druid/single-server/small/_common/log4j2.xml index 0bda2f6968f..eb24a2a8940 100644 --- a/examples/conf/druid/single-server/small/_common/log4j2.xml +++ b/examples/conf/druid/single-server/small/_common/log4j2.xml @@ -82,5 +82,10 @@ + + + + + diff --git a/examples/conf/druid/single-server/xlarge/_common/log4j2.xml b/examples/conf/druid/single-server/xlarge/_common/log4j2.xml index 0bda2f6968f..eb24a2a8940 100644 --- a/examples/conf/druid/single-server/xlarge/_common/log4j2.xml +++ b/examples/conf/druid/single-server/xlarge/_common/log4j2.xml @@ -82,5 +82,10 @@ + + + + + From cfd07a95b7d592a333ca51597e9bbdd68e18a88a Mon Sep 17 00:00:00 2001 From: imply-cheddar <86940447+imply-cheddar@users.noreply.github.com> Date: Mon, 19 Jun 2023 17:11:13 +0900 Subject: [PATCH 06/74] Errors take 3 (#14004) Introduce DruidException, an exception whose goal in life is to be delivered to a user. DruidException itself has javadoc on it to describe how it should be used. This commit both introduces the Exception and adjusts some of the places that are generating exceptions to generate DruidException objects instead, as a way to show how the Exception should be used. This work was a 3rd iteration on top of work that was started by Paul Rogers. I don't know if his name will survive the squash-and-merge, so I'm calling it out here and thanking him for starting on this. --- .../druid/benchmark/query/SqlBenchmark.java | 4 +- .../query/SqlExpressionBenchmark.java | 3 +- .../query/SqlNestedDataBenchmark.java | 2 +- .../benchmark/query/SqlVsNativeBenchmark.java | 2 +- extensions-core/datasketches/pom.xml | 5 + .../sql/ThetaSketchSqlAggregatorTest.java | 2 +- ...ArrayOfDoublesSketchSqlAggregatorTest.java | 316 ++++++------ .../druid/msq/sql/MSQTaskSqlEngine.java | 12 +- .../apache/druid/msq/sql/SqlTaskResource.java | 26 +- .../apache/druid/msq/exec/MSQInsertTest.java | 63 ++- .../apache/druid/msq/exec/MSQReplaceTest.java | 43 +- .../apache/druid/msq/exec/MSQSelectTest.java | 48 +- ...iaryDataManagerManualAddAndDeleteTest.java | 54 ++- .../druid/tests/query/ITJdbcQueryTest.java | 4 +- .../common/exception/DruidException.java | 4 + .../apache/druid/common/utils/IdUtils.java | 59 ++- .../druid/data/input/impl/JsonNodeReader.java | 2 +- .../apache/druid/error/DruidException.java | 449 ++++++++++++++++++ .../org/apache/druid/error/ErrorResponse.java | 214 +++++++++ .../org/apache/druid/error/InvalidInput.java | 63 +++ .../apache/druid/error/InvalidSqlInput.java | 50 ++ .../druid/error/QueryExceptionCompat.java | 79 +++ .../druid/java/util/common/logger/Logger.java | 31 ++ .../segment/nested/NestedPathFinder.java | 35 +- .../druid/common/utils/IdUtilsTest.java | 72 ++- .../druid/error/DruidExceptionMatcher.java | 126 +++++ .../apache/druid/error/ErrorResponseTest.java | 110 +++++ .../apache/druid/matchers/DruidMatchers.java | 49 ++ .../apache/druid/matchers/LambdaMatcher.java | 62 +++ .../segment/nested/NestedPathFinderTest.java | 73 ++- .../apache/druid/server/QueryResource.java | 45 -- .../druid/server/QueryResultPusher.java | 83 ++-- .../security/AllowOptionsResourceFilter.java | 2 - .../PreResponseAuthorizationCheckFilter.java | 2 +- .../segment/indexing/DataSchemaTest.java | 100 ++-- .../druid/server/QueryResourceTest.java | 248 ++++++++-- .../server/security/AuthValidatorTest.java | 37 +- sql/pom.xml | 5 + sql/src/main/codegen/includes/common.ftl | 1 - sql/src/main/codegen/includes/insert.ftl | 4 +- sql/src/main/codegen/includes/replace.ftl | 4 +- .../apache/druid/sql/AbstractStatement.java | 14 +- .../org/apache/druid/sql/DirectStatement.java | 22 +- .../aggregation/builtin/AvgSqlAggregator.java | 8 +- .../EarliestLatestAnySqlAggregator.java | 79 ++- .../aggregation/builtin/MaxSqlAggregator.java | 12 +- .../aggregation/builtin/MinSqlAggregator.java | 3 +- .../builtin/SimpleSqlAggregator.java | 8 + .../builtin/StringSqlAggregator.java | 13 +- .../aggregation/builtin/SumSqlAggregator.java | 10 +- .../NestedDataOperatorConversions.java | 60 +-- .../calcite/parser/DruidSqlParserUtils.java | 301 ++++++------ .../sql/calcite/planner/DruidPlanner.java | 232 ++++++++- .../sql/calcite/planner/DruidRexExecutor.java | 31 +- .../sql/calcite/planner/IngestHandler.java | 121 +++-- .../sql/calcite/planner/PlannerFactory.java | 9 +- .../sql/calcite/planner/QueryHandler.java | 155 +++--- .../planner/RelParameterizerShuttle.java | 20 +- .../planner/SqlParameterizerShuttle.java | 33 +- .../calcite/planner/SqlStatementHandler.java | 5 +- .../sql/calcite/rel/DruidJoinQueryRel.java | 7 +- .../druid/sql/calcite/rel/DruidQuery.java | 12 +- .../calcite/rule/DruidLogicalValuesRule.java | 12 +- .../sql/calcite/run/NativeSqlEngine.java | 15 +- .../druid/sql/calcite/run/SqlEngine.java | 2 +- .../druid/sql/calcite/run/SqlEngines.java | 15 +- .../sql/calcite/table/RowSignatures.java | 5 + .../apache/druid/sql/http/SqlResource.java | 41 +- .../apache/druid/sql/SqlStatementTest.java | 143 +++--- .../sql/avatica/DruidAvaticaHandlerTest.java | 242 ++++++---- .../druid/sql/avatica/DruidStatementTest.java | 19 +- .../sql/calcite/BaseCalciteQueryTest.java | 46 +- .../calcite/CalciteCorrelatedQueryTest.java | 1 - .../sql/calcite/CalciteIngestionDmlTest.java | 11 +- .../sql/calcite/CalciteInsertDmlTest.java | 146 +++--- .../sql/calcite/CalciteJoinQueryTest.java | 50 +- .../CalciteMultiValueStringQueryTest.java | 15 +- .../calcite/CalciteNestedDataQueryTest.java | 10 +- .../calcite/CalciteParameterQueryTest.java | 17 +- .../druid/sql/calcite/CalciteQueryTest.java | 260 +++++----- .../sql/calcite/CalciteReplaceDmlTest.java | 127 ++--- .../sql/calcite/CalciteSelectQueryTest.java | 16 +- .../druid/sql/calcite/QueryTestRunner.java | 4 +- .../SqlVectorizedExpressionSanityTest.java | 5 +- .../parser/DruidSqlParserUtilsTest.java | 46 +- .../rule/DruidLogicalValuesRuleTest.java | 29 +- .../druid/sql/calcite/util/QueryLogHook.java | 17 + .../druid/sql/http/SqlResourceTest.java | 425 +++++++++++------ 88 files changed, 3701 insertions(+), 1766 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/error/DruidException.java create mode 100644 processing/src/main/java/org/apache/druid/error/ErrorResponse.java create mode 100644 processing/src/main/java/org/apache/druid/error/InvalidInput.java create mode 100644 processing/src/main/java/org/apache/druid/error/InvalidSqlInput.java create mode 100644 processing/src/main/java/org/apache/druid/error/QueryExceptionCompat.java create mode 100644 processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java create mode 100644 processing/src/test/java/org/apache/druid/error/ErrorResponseTest.java create mode 100644 processing/src/test/java/org/apache/druid/matchers/DruidMatchers.java create mode 100644 processing/src/test/java/org/apache/druid/matchers/LambdaMatcher.java diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 9459b234303..ba9debb2881 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -543,7 +543,7 @@ public class SqlBenchmark @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void querySql(Blackhole blackhole) throws Exception + public void querySql(Blackhole blackhole) { final Map context = ImmutableMap.of( QueryContexts.VECTORIZE_KEY, vectorize, @@ -561,7 +561,7 @@ public class SqlBenchmark @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void planSql(Blackhole blackhole) throws Exception + public void planSql(Blackhole blackhole) { final Map context = ImmutableMap.of( QueryContexts.VECTORIZE_KEY, vectorize, diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java index 1c64d7a749d..7733281908f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java @@ -67,7 +67,6 @@ import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; import javax.annotation.Nullable; - import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -349,7 +348,7 @@ public class SqlExpressionBenchmark @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void querySql(Blackhole blackhole) throws Exception + public void querySql(Blackhole blackhole) { final Map context = ImmutableMap.of( QueryContexts.VECTORIZE_KEY, vectorize, diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java index aeda68b25b0..98514512e9a 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java @@ -375,7 +375,7 @@ public class SqlNestedDataBenchmark @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void querySql(Blackhole blackhole) throws Exception + public void querySql(Blackhole blackhole) { final Map context = ImmutableMap.of( QueryContexts.VECTORIZE_KEY, vectorize, diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java index 1cb747048dc..de3db00accf 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java @@ -170,7 +170,7 @@ public class SqlVsNativeBenchmark @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void queryPlanner(Blackhole blackhole) throws Exception + public void queryPlanner(Blackhole blackhole) { try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sqlQuery, Collections.emptyMap())) { final PlannerResult plannerResult = planner.plan(); diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index 2657926a8a3..e5d408256aa 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -165,6 +165,11 @@ hamcrest-core test + + org.hamcrest + hamcrest-all + test + nl.jqno.equalsverifier equalsverifier diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java index 0887cc4e69a..c1ddfa279d2 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java @@ -1035,7 +1035,7 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest { assertQueryIsUnplannable( "SELECT THETA_SKETCH_INTERSECT(NULL, NULL) FROM foo", - "Possible error: THETA_SKETCH_INTERSECT can only be used on aggregates. " + + "THETA_SKETCH_INTERSECT can only be used on aggregates. " + "It cannot be used directly on a column or on a scalar expression." ); } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java index 3be8e8b1885..a240f89bdcc 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java @@ -158,13 +158,13 @@ public class ArrayOfDoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest cannotVectorize(); final String sql = "SELECT\n" - + " dim1,\n" - + " SUM(cnt),\n" - + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES(tuplesketch_dim2)),\n" - + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES(dim2, m1)),\n" - + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES(dim2, m1, 256))\n" - + "FROM druid.foo\n" - + "GROUP BY dim1"; + + " dim1,\n" + + " SUM(cnt),\n" + + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES(tuplesketch_dim2)),\n" + + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES(dim2, m1)),\n" + + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES(dim2, m1, 256))\n" + + "FROM druid.foo\n" + + "GROUP BY dim1"; final List expectedResults; @@ -189,54 +189,54 @@ public class ArrayOfDoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest sql, ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING)) - .setAggregatorSpecs( - aggregators( - new LongSumAggregatorFactory("a0", "cnt"), - new ArrayOfDoublesSketchAggregatorFactory( - "a1", - "tuplesketch_dim2", - null, - null, - null - ), - new ArrayOfDoublesSketchAggregatorFactory( - "a2", - "dim2", - null, - ImmutableList.of("m1"), - null - ), - new ArrayOfDoublesSketchAggregatorFactory( - "a3", - "dim2", - 256, - ImmutableList.of("m1"), - null - ) - ) - ) - .setPostAggregatorSpecs( - ImmutableList.of( - new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( - "p1", - new FieldAccessPostAggregator("p0", "a1") - ), - new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( - "p3", - new FieldAccessPostAggregator("p2", "a2") - ), - new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( - "p5", - new FieldAccessPostAggregator("p4", "a3") - ) - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING)) + .setAggregatorSpecs( + aggregators( + new LongSumAggregatorFactory("a0", "cnt"), + new ArrayOfDoublesSketchAggregatorFactory( + "a1", + "tuplesketch_dim2", + null, + null, + null + ), + new ArrayOfDoublesSketchAggregatorFactory( + "a2", + "dim2", + null, + ImmutableList.of("m1"), + null + ), + new ArrayOfDoublesSketchAggregatorFactory( + "a3", + "dim2", + 256, + ImmutableList.of("m1"), + null + ) + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( + "p1", + new FieldAccessPostAggregator("p0", "a1") + ), + new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( + "p3", + new FieldAccessPostAggregator("p2", "a2") + ), + new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( + "p5", + new FieldAccessPostAggregator("p4", "a3") + ) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() ), expectedResults ); @@ -248,14 +248,14 @@ public class ArrayOfDoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest cannotVectorize(); final String sql = "SELECT\n" - + " SUM(cnt),\n" - + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES(tuplesketch_dim2)) AS all_sum_estimates,\n" - + StringUtils.replace( - "DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES_INTERSECT(COMPLEX_DECODE_BASE64('arrayOfDoublesSketch', '%s'), DS_TUPLE_DOUBLES(tuplesketch_dim2), 128)) AS intersect_sum_estimates\n", - "%s", - COMPACT_BASE_64_ENCODED_SKETCH_FOR_INTERSECTION - ) - + "FROM druid.foo"; + + " SUM(cnt),\n" + + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES(tuplesketch_dim2)) AS all_sum_estimates,\n" + + StringUtils.replace( + "DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(DS_TUPLE_DOUBLES_INTERSECT(COMPLEX_DECODE_BASE64('arrayOfDoublesSketch', '%s'), DS_TUPLE_DOUBLES(tuplesketch_dim2), 128)) AS intersect_sum_estimates\n", + "%s", + COMPACT_BASE_64_ENCODED_SKETCH_FOR_INTERSECTION + ) + + "FROM druid.foo"; final List expectedResults; @@ -268,8 +268,12 @@ public class ArrayOfDoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest ); final String expectedBase64Constant = "'" - + StringUtils.replace(COMPACT_BASE_64_ENCODED_SKETCH_FOR_INTERSECTION, "=", "\\u003D") - + "'"; + + StringUtils.replace( + COMPACT_BASE_64_ENCODED_SKETCH_FOR_INTERSECTION, + "=", + "\\u003D" + ) + + "'"; testQuery( sql, @@ -282,38 +286,40 @@ public class ArrayOfDoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest ImmutableList.of( new LongSumAggregatorFactory("a0", "cnt"), new ArrayOfDoublesSketchAggregatorFactory( - "a1", - "tuplesketch_dim2", - null, - null, - null + "a1", + "tuplesketch_dim2", + null, + null, + null ) ) ) .postAggregators( ImmutableList.of( - new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( - "p1", - new FieldAccessPostAggregator("p0", "a1") - ), - new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( - "p5", - new ArrayOfDoublesSketchSetOpPostAggregator( - "p4", - "INTERSECT", - 128, - null, - ImmutableList.of( - new ExpressionPostAggregator( - "p2", - "complex_decode_base64('arrayOfDoublesSketch'," + expectedBase64Constant + ")", - null, - queryFramework().macroTable() - ), - new FieldAccessPostAggregator("p3", "a1") - ) - ) - ) + new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( + "p1", + new FieldAccessPostAggregator("p0", "a1") + ), + new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( + "p5", + new ArrayOfDoublesSketchSetOpPostAggregator( + "p4", + "INTERSECT", + 128, + null, + ImmutableList.of( + new ExpressionPostAggregator( + "p2", + "complex_decode_base64('arrayOfDoublesSketch'," + + expectedBase64Constant + + ")", + null, + queryFramework().macroTable() + ), + new FieldAccessPostAggregator("p3", "a1") + ) + ) + ) ) ) .context(QUERY_CONTEXT_DEFAULT) @@ -329,12 +335,12 @@ public class ArrayOfDoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest cannotVectorize(); final String sql = "SELECT\n" - + " DS_TUPLE_DOUBLES(NULL),\n" - + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(NULL),\n" - + " DS_TUPLE_DOUBLES_UNION(NULL, NULL),\n" - + " DS_TUPLE_DOUBLES_UNION(NULL, DS_TUPLE_DOUBLES(tuplesketch_dim2)),\n" - + " DS_TUPLE_DOUBLES_UNION(DS_TUPLE_DOUBLES(tuplesketch_dim2), NULL)\n" - + "FROM druid.foo"; + + " DS_TUPLE_DOUBLES(NULL),\n" + + " DS_TUPLE_DOUBLES_METRICS_SUM_ESTIMATE(NULL),\n" + + " DS_TUPLE_DOUBLES_UNION(NULL, NULL),\n" + + " DS_TUPLE_DOUBLES_UNION(NULL, DS_TUPLE_DOUBLES(tuplesketch_dim2)),\n" + + " DS_TUPLE_DOUBLES_UNION(DS_TUPLE_DOUBLES(tuplesketch_dim2), NULL)\n" + + "FROM druid.foo"; final List expectedResults; @@ -345,7 +351,7 @@ public class ArrayOfDoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest "\"AQEJAwQBzJP/////////fw==\"", "\"AQEJAwgBzJP/////////fwIAAAAAAAAAjFnadZuMrkg6WYAWZ8t1NgAAAAAAACBAAAAAAAAANkA=\"", "\"AQEJAwgBzJP/////////fwIAAAAAAAAAjFnadZuMrkg6WYAWZ8t1NgAAAAAAACBAAAAAAAAANkA=\"", - } + } ); testQuery( @@ -366,57 +372,57 @@ public class ArrayOfDoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest .aggregators( ImmutableList.of( new ArrayOfDoublesSketchAggregatorFactory( - "a0", - "v0", - null, - null, - null + "a0", + "v0", + null, + null, + null ), new ArrayOfDoublesSketchAggregatorFactory( - "a1", - "tuplesketch_dim2", - null, - null, - null + "a1", + "tuplesketch_dim2", + null, + null, + null ) ) ) .postAggregators( ImmutableList.of( - new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( - "p1", - new ExpressionPostAggregator("p0", "null", null, queryFramework().macroTable()) - ), - new ArrayOfDoublesSketchSetOpPostAggregator( - "p4", - ArrayOfDoublesSketchOperations.Operation.UNION.name(), - null, - null, - ImmutableList.of( - new ExpressionPostAggregator("p2", "null", null, queryFramework().macroTable()), - new ExpressionPostAggregator("p3", "null", null, queryFramework().macroTable()) - ) - ), - new ArrayOfDoublesSketchSetOpPostAggregator( - "p7", - ArrayOfDoublesSketchOperations.Operation.UNION.name(), - null, - null, - ImmutableList.of( - new ExpressionPostAggregator("p5", "null", null, queryFramework().macroTable()), - new FieldAccessPostAggregator("p6", "a1") - ) - ), - new ArrayOfDoublesSketchSetOpPostAggregator( - "p10", - ArrayOfDoublesSketchOperations.Operation.UNION.name(), - null, - null, - ImmutableList.of( - new FieldAccessPostAggregator("p8", "a1"), - new ExpressionPostAggregator("p9", "null", null, queryFramework().macroTable()) - ) - ) + new ArrayOfDoublesSketchToMetricsSumEstimatePostAggregator( + "p1", + new ExpressionPostAggregator("p0", "null", null, queryFramework().macroTable()) + ), + new ArrayOfDoublesSketchSetOpPostAggregator( + "p4", + ArrayOfDoublesSketchOperations.Operation.UNION.name(), + null, + null, + ImmutableList.of( + new ExpressionPostAggregator("p2", "null", null, queryFramework().macroTable()), + new ExpressionPostAggregator("p3", "null", null, queryFramework().macroTable()) + ) + ), + new ArrayOfDoublesSketchSetOpPostAggregator( + "p7", + ArrayOfDoublesSketchOperations.Operation.UNION.name(), + null, + null, + ImmutableList.of( + new ExpressionPostAggregator("p5", "null", null, queryFramework().macroTable()), + new FieldAccessPostAggregator("p6", "a1") + ) + ), + new ArrayOfDoublesSketchSetOpPostAggregator( + "p10", + ArrayOfDoublesSketchOperations.Operation.UNION.name(), + null, + null, + ImmutableList.of( + new FieldAccessPostAggregator("p8", "a1"), + new ExpressionPostAggregator("p9", "null", null, queryFramework().macroTable()) + ) + ) ) ) .context(QUERY_CONTEXT_DEFAULT) @@ -429,24 +435,30 @@ public class ArrayOfDoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest @Test public void testArrayOfDoublesSketchIntersectOnScalarExpression() { - assertQueryIsUnplannable("SELECT DS_TUPLE_DOUBLES_INTERSECT(NULL, NULL) FROM foo", - "Possible error: DS_TUPLE_DOUBLES_INTERSECT can only be used on aggregates. " + - "It cannot be used directly on a column or on a scalar expression."); + assertQueryIsUnplannable( + "SELECT DS_TUPLE_DOUBLES_INTERSECT(NULL, NULL) FROM foo", + "DS_TUPLE_DOUBLES_INTERSECT can only be used on aggregates. " + + "It cannot be used directly on a column or on a scalar expression." + ); } @Test public void testArrayOfDoublesSketchNotOnScalarExpression() { - assertQueryIsUnplannable("SELECT DS_TUPLE_DOUBLES_NOT(NULL, NULL) FROM foo", - "Possible error: DS_TUPLE_DOUBLES_NOT can only be used on aggregates. " + - "It cannot be used directly on a column or on a scalar expression."); + assertQueryIsUnplannable( + "SELECT DS_TUPLE_DOUBLES_NOT(NULL, NULL) FROM foo", + "DS_TUPLE_DOUBLES_NOT can only be used on aggregates. " + + "It cannot be used directly on a column or on a scalar expression." + ); } @Test public void testArrayOfDoublesSketchUnionOnScalarExpression() { - assertQueryIsUnplannable("SELECT DS_TUPLE_DOUBLES_UNION(NULL, NULL) FROM foo", - "Possible error: DS_TUPLE_DOUBLES_UNION can only be used on aggregates. " + - "It cannot be used directly on a column or on a scalar expression."); + assertQueryIsUnplannable( + "SELECT DS_TUPLE_DOUBLES_UNION(NULL, NULL) FROM foo", + "DS_TUPLE_DOUBLES_UNION can only be used on aggregates. " + + "It cannot be used directly on a column or on a scalar expression." + ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java index 94c2532ca79..7d27103e0b9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java @@ -32,6 +32,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.tools.ValidationException; import org.apache.calcite.util.Pair; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -86,7 +87,7 @@ public class MSQTaskSqlEngine implements SqlEngine } @Override - public void validateContext(Map queryContext) throws ValidationException + public void validateContext(Map queryContext) { SqlEngines.validateNoSpecialContextKeys(queryContext, SYSTEM_CONTEXT_PARAMETERS); } @@ -250,18 +251,17 @@ public class MSQTaskSqlEngine implements SqlEngine * queries, because we use these output names to generate columns in segments. They must be unique. */ private static void validateNoDuplicateAliases(final List> fieldMappings) - throws ValidationException { final Set aliasesSeen = new HashSet<>(); for (final Pair field : fieldMappings) { if (!aliasesSeen.add(field.right)) { - throw new ValidationException("Duplicate field in SELECT: [" + field.right + "]"); + throw InvalidSqlInput.exception("Duplicate field in SELECT: [%s]", field.right); } } } - private static void validateLimitAndOffset(final RelNode topRel, final boolean limitOk) throws ValidationException + private static void validateLimitAndOffset(final RelNode topRel, final boolean limitOk) { Sort sort = null; @@ -283,13 +283,13 @@ public class MSQTaskSqlEngine implements SqlEngine // The segment generator relies on shuffle statistics to determine segment intervals when PARTITIONED BY is not ALL, // and LIMIT/OFFSET prevent shuffle statistics from being generated. This is because they always send everything // to a single partition, so there are no shuffle statistics. - throw new ValidationException( + throw InvalidSqlInput.exception( "INSERT and REPLACE queries cannot have a LIMIT unless PARTITIONED BY is \"ALL\"." ); } if (sort != null && sort.offset != null) { // Found an outer OFFSET that is not allowed. - throw new ValidationException("INSERT and REPLACE queries cannot have an OFFSET."); + throw InvalidSqlInput.exception("INSERT and REPLACE queries cannot have an OFFSET."); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java index f0cd7318f64..d270963db46 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java @@ -23,8 +23,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.io.CountingOutputStream; import com.google.inject.Inject; -import org.apache.calcite.plan.RelOptPlanner; import org.apache.druid.common.exception.SanitizableException; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.ErrorResponse; import org.apache.druid.guice.annotations.MSQ; import org.apache.druid.indexer.TaskState; import org.apache.druid.java.util.common.guava.Sequence; @@ -69,15 +70,15 @@ import java.util.Collections; /** * Endpoint for SQL execution using MSQ tasks. - * + *

* Unlike the SQL endpoint in {@link SqlResource}, this endpoint returns task IDs instead of inline results. Queries * are executed asynchronously using MSQ tasks via the indexing service (Overlord + MM or Indexer). This endpoint * does not provide a way for users to get the status or results of a query. That must be done using Overlord APIs * for status and reports. - * + *

* One exception: EXPLAIN query results are returned inline by this endpoint, in the same way as {@link SqlResource} * would return them. - * + *

* This endpoint does not support system tables or INFORMATION_SCHEMA. Queries on those tables result in errors. */ @Path("/druid/v2/sql/task/") @@ -129,7 +130,7 @@ public class SqlTaskResource /** * Post a query task. - * + *

* Execution uses {@link MSQTaskSqlEngine} to ship the query off to the Overlord as an indexing task using * {@link org.apache.druid.msq.indexing.MSQControllerTask}. The task ID is returned immediately to the caller, * and execution proceeds asynchronously. @@ -159,6 +160,13 @@ public class SqlTaskResource return buildStandardResponse(sequence, sqlQuery, sqlQueryId, rowTransformer); } } + catch (DruidException e) { + stmt.reporter().failed(e); + return Response.status(e.getStatusCode()) + .type(MediaType.APPLICATION_JSON_TYPE) + .entity(new ErrorResponse(e)) + .build(); + } // Kitchen-sinking the errors since they are all unchecked. // Just copied from SqlResource. catch (QueryCapacityExceededException cap) { @@ -182,14 +190,6 @@ public class SqlTaskResource throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() .transformIfNeeded(e); // let ForbiddenExceptionMapper handle this } - catch (RelOptPlanner.CannotPlanException e) { - stmt.reporter().failed(e); - SqlPlanningException spe = new SqlPlanningException( - SqlPlanningException.PlanningError.UNSUPPORTED_SQL_ERROR, - e.getMessage() - ); - return buildNonOkResponse(BadQueryException.STATUS_CODE, spe, sqlQueryId); - } // Calcite throws a java.lang.AssertionError which is type Error not Exception. Using Throwable catches both. catch (Throwable e) { stmt.reporter().failed(e); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index b55de6c165c..1769c3028a5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -43,7 +44,6 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFact import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.timeline.SegmentId; import org.apache.druid.utils.CompressionUtils; import org.hamcrest.CoreMatchers; @@ -92,7 +92,6 @@ public class MSQInsertTest extends MSQTestBase @Parameterized.Parameter(1) public Map context; - @Test public void testInsertOnFoo1() { @@ -541,11 +540,9 @@ public class MSQInsertTest extends MSQTestBase "INSERT INTO foo1 SELECT count(dim3) FROM foo WHERE dim3 IS NOT NULL GROUP BY 1 PARTITIONED BY ALL TIME") .setExpectedDataSource("foo1") .setQueryContext(context) - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "Aggregate expression is illegal in GROUP BY clause")) - )) + .setExpectedValidationErrorMatcher( + invalidSqlContains("Aggregate expression is illegal in GROUP BY clause") + ) .verifyPlanningErrors(); } @@ -747,11 +744,9 @@ public class MSQInsertTest extends MSQTestBase + "PARTITIONED BY DAY " + "CLUSTERED BY dim1 DESC" ) - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( - "[`dim1` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order.")) - )) + .setExpectedValidationErrorMatcher( + invalidSqlIs("Invalid CLUSTERED BY clause [`dim1` DESC]: cannot sort in descending order.") + ) .verifyPlanningErrors(); } @@ -967,7 +962,7 @@ public class MSQInsertTest extends MSQTestBase .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), + CoreMatchers.instanceOf(DruidException.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( "Field \"__time\" must be of type TIMESTAMP")) )) @@ -977,14 +972,14 @@ public class MSQInsertTest extends MSQTestBase @Test public void testIncorrectInsertQuery() { - testIngestQuery().setSql( - "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 clustered by dim1") - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( - "CLUSTERED BY found before PARTITIONED BY. In Druid, the CLUSTERED BY clause must follow the PARTITIONED BY clause")) - )) - .verifyPlanningErrors(); + testIngestQuery() + .setSql( + "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 clustered by dim1" + ) + .setExpectedValidationErrorMatcher(invalidSqlContains( + "CLUSTERED BY found before PARTITIONED BY, CLUSTERED BY must come after the PARTITIONED BY clause" + )) + .verifyPlanningErrors(); } @@ -1032,11 +1027,9 @@ public class MSQInsertTest extends MSQTestBase + " )\n" + ") PARTITIONED by day") .setQueryContext(context) - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "Duplicate field in SELECT: [namespace]")) - )) + .setExpectedValidationErrorMatcher( + invalidSqlIs("Duplicate field in SELECT: [namespace]") + ) .verifyPlanningErrors(); } @@ -1097,11 +1090,11 @@ public class MSQInsertTest extends MSQTestBase + "FROM foo " + "LIMIT 50 " + "PARTITIONED BY MONTH") - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "INSERT and REPLACE queries cannot have a LIMIT unless PARTITIONED BY is \"ALL\"")) - )) + .setExpectedValidationErrorMatcher( + invalidSqlContains( + "INSERT and REPLACE queries cannot have a LIMIT unless PARTITIONED BY is \"ALL\"" + ) + ) .setQueryContext(context) .verifyPlanningErrors(); } @@ -1115,11 +1108,9 @@ public class MSQInsertTest extends MSQTestBase + "LIMIT 50 " + "OFFSET 10" + "PARTITIONED BY ALL TIME") - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "INSERT and REPLACE queries cannot have an OFFSET")) - )) + .setExpectedValidationErrorMatcher( + invalidSqlContains("INSERT and REPLACE queries cannot have an OFFSET") + ) .setQueryContext(context) .verifyPlanningErrors(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 1dfd7742146..500d2a68bee 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -30,13 +30,10 @@ import org.apache.druid.msq.test.MSQTestFileUtils; import org.apache.druid.msq.test.MSQTestTaskActionClient; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; -import org.hamcrest.CoreMatchers; import org.junit.Test; -import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.mockito.ArgumentMatchers; @@ -327,17 +324,15 @@ public class MSQReplaceTest extends MSQTestBase @Test public void testReplaceIncorrectSyntax() { - testIngestQuery().setSql("REPLACE INTO foo1 OVERWRITE SELECT * FROM foo PARTITIONED BY ALL TIME") - .setExpectedDataSource("foo1") - .setQueryContext(context) - .setExpectedValidationErrorMatcher( - CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "Missing time chunk information in OVERWRITE clause for REPLACE. Use OVERWRITE WHERE <__time based condition> or OVERWRITE ALL to overwrite the entire table.")) - ) - ) - .verifyPlanningErrors(); + testIngestQuery() + .setSql("REPLACE INTO foo1 OVERWRITE SELECT * FROM foo PARTITIONED BY ALL TIME") + .setExpectedDataSource("foo1") + .setQueryContext(context) + .setExpectedValidationErrorMatcher(invalidSqlContains( + "Missing time chunk information in OVERWRITE clause for REPLACE. " + + "Use OVERWRITE WHERE <__time based condition> or OVERWRITE ALL to overwrite the entire table." + )) + .verifyPlanningErrors(); } @Test @@ -581,10 +576,8 @@ public class MSQReplaceTest extends MSQTestBase + "LIMIT 50" + "PARTITIONED BY MONTH") .setQueryContext(context) - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "INSERT and REPLACE queries cannot have a LIMIT unless PARTITIONED BY is \"ALL\"")) + .setExpectedValidationErrorMatcher(invalidSqlContains( + "INSERT and REPLACE queries cannot have a LIMIT unless PARTITIONED BY is \"ALL\"" )) .verifyPlanningErrors(); } @@ -599,10 +592,8 @@ public class MSQReplaceTest extends MSQTestBase + "LIMIT 50 " + "OFFSET 10" + "PARTITIONED BY ALL TIME") - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "INSERT and REPLACE queries cannot have an OFFSET")) + .setExpectedValidationErrorMatcher(invalidSqlContains( + "INSERT and REPLACE queries cannot have an OFFSET" )) .setQueryContext(context) .verifyPlanningErrors(); @@ -742,11 +733,9 @@ public class MSQReplaceTest extends MSQTestBase + "PARTITIONED BY ALL TIME " + "CLUSTERED BY m2, m1 DESC" ) - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( - "[`m1` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order.")) - )) + .setExpectedValidationErrorMatcher( + invalidSqlIs("Invalid CLUSTERED BY clause [`m1` DESC]: cannot sort in descending order.") + ) .verifyPlanningErrors(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index b09e3aa01f1..0ad6e4d90be 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -25,6 +25,8 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -64,7 +66,6 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.external.ExternalDataSource; import org.apache.druid.sql.calcite.filtration.Filtration; @@ -72,7 +73,6 @@ import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.planner.JoinAlgorithm; import org.apache.druid.sql.calcite.planner.PlannerContext; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.apache.druid.sql.calcite.util.CalciteTests; import org.hamcrest.CoreMatchers; import org.junit.Test; @@ -95,7 +95,6 @@ import java.util.Map; @RunWith(Parameterized.class) public class MSQSelectTest extends MSQTestBase { - @Parameterized.Parameters(name = "{index}:with context {0}") public static Collection data() { @@ -1186,10 +1185,9 @@ public class MSQSelectTest extends MSQTestBase { testSelectQuery() .setSql("select a from ") - .setExpectedValidationErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Encountered \"from \"")) - )) + .setExpectedValidationErrorMatcher( + invalidSqlContains("Received an unexpected token [from ] (line [1], column [10]), acceptable options") + ) .setQueryContext(context) .verifyPlanningErrors(); } @@ -1201,11 +1199,7 @@ public class MSQSelectTest extends MSQTestBase .setSql("SELECT * FROM INFORMATION_SCHEMA.SCHEMATA") .setQueryContext(context) .setExpectedValidationErrorMatcher( - CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( - "Cannot query table INFORMATION_SCHEMA.SCHEMATA with SQL engine 'msq-task'.")) - ) + invalidSqlIs("Cannot query table(s) [INFORMATION_SCHEMA.SCHEMATA] with SQL engine [msq-task]") ) .verifyPlanningErrors(); } @@ -1217,11 +1211,7 @@ public class MSQSelectTest extends MSQTestBase .setSql("SELECT * FROM sys.segments") .setQueryContext(context) .setExpectedValidationErrorMatcher( - CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( - "Cannot query table sys.segments with SQL engine 'msq-task'.")) - ) + invalidSqlIs("Cannot query table(s) [sys.segments] with SQL engine [msq-task]") ) .verifyPlanningErrors(); } @@ -1233,11 +1223,7 @@ public class MSQSelectTest extends MSQTestBase .setSql("select s.segment_id, s.num_rows, f.dim1 from sys.segments as s, foo as f") .setQueryContext(context) .setExpectedValidationErrorMatcher( - CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( - "Cannot query table sys.segments with SQL engine 'msq-task'.")) - ) + invalidSqlIs("Cannot query table(s) [sys.segments] with SQL engine [msq-task]") ) .verifyPlanningErrors(); } @@ -1250,16 +1236,11 @@ public class MSQSelectTest extends MSQTestBase + "select segment_source.segment_id, segment_source.num_rows from segment_source") .setQueryContext(context) .setExpectedValidationErrorMatcher( - CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith( - "Cannot query table sys.segments with SQL engine 'msq-task'.")) - ) + invalidSqlIs("Cannot query table(s) [sys.segments] with SQL engine [msq-task]") ) .verifyPlanningErrors(); } - @Test public void testSelectOnUserDefinedSourceContainingWith() { @@ -1644,8 +1625,13 @@ public class MSQSelectTest extends MSQTestBase + "FROM kttm_data " + "GROUP BY 1") .setExpectedValidationErrorMatcher( - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "LATEST() aggregator depends on __time column")) + new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general") + .expectMessageIs( + "Query planning failed for unknown reason, our best guess is this " + + "[LATEST and EARLIEST aggregators implicitly depend on the __time column, " + + "but the table queried doesn't contain a __time column. " + + "Please use LATEST_BY or EARLIEST_BY and specify the column explicitly.]" + ) ) .setExpectedRowSignature(rowSignature) .verifyPlanningErrors(); @@ -1676,7 +1662,7 @@ public class MSQSelectTest extends MSQTestBase .setSql("select unique_dim1 from foo2 group by unique_dim1") .setQueryContext(context) .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(UnsupportedSQLQueryException.class), + CoreMatchers.instanceOf(DruidException.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( "SQL requires a group-by on a column of type COMPLEX that is unsupported")) )) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java index 75fed3e81dd..fabb1cfb196 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java @@ -24,6 +24,8 @@ import com.google.common.io.ByteSource; import com.google.common.primitives.Ints; import org.apache.commons.io.FileUtils; import org.apache.druid.client.indexing.NoopOverlordClient; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.worker.config.WorkerConfig; @@ -36,13 +38,13 @@ import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.BuildingShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpecLookup; +import org.hamcrest.MatcherAssert; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import java.io.File; @@ -56,9 +58,6 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest @Rule public TemporaryFolder tempDir = new TemporaryFolder(); - @Rule - public ExpectedException expectedException = ExpectedException.none(); - private LocalIntermediaryDataManager intermediaryDataManager; private File intermediarySegmentsLocation; private File siblingLocation; @@ -93,11 +92,14 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest DataSegment segment = newSegment(Intervals.of("2018/2019"), i); intermediaryDataManager.addSegment("supervisorTaskId", "subTaskId", segment, segmentFile); } - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("Can't find location to handle segment"); File segmentFile = generateSegmentDir("file_" + i); DataSegment segment = newSegment(Intervals.of("2018/2019"), 4); - intermediaryDataManager.addSegment("supervisorTaskId", "subTaskId", segment, segmentFile); + + IllegalStateException e = Assert.assertThrows( + IllegalStateException.class, + () -> intermediaryDataManager.addSegment("supervisorTaskId", "subTaskId", segment, segmentFile) + ); + Assert.assertEquals(StringUtils.format("Can't find location to handle segment[%s]", segment), e.getMessage()); } @Test @@ -140,7 +142,8 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest for (int partitionId = 0; partitionId < 2; partitionId++) { for (int subTaskId = 0; subTaskId < 2; subTaskId++) { Assert.assertFalse( - intermediaryDataManager.findPartitionFile(supervisorTaskId, "subTaskId_" + subTaskId, interval, partitionId).isPresent() + intermediaryDataManager.findPartitionFile(supervisorTaskId, "subTaskId_" + subTaskId, interval, partitionId) + .isPresent() ); } } @@ -166,8 +169,6 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest @Test public void testFailsWithCraftyFabricatedNamesForDelete() throws IOException { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("supervisorTaskId cannot start with the '.' character."); final String supervisorTaskId = "../" + siblingLocation.getName(); final String someFile = "sneaky-snake.txt"; File dataFile = new File(siblingLocation, someFile); @@ -178,7 +179,15 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest ); Assert.assertTrue(new File(intermediarySegmentsLocation, supervisorTaskId).exists()); Assert.assertTrue(dataFile.exists()); - intermediaryDataManager.deletePartitions(supervisorTaskId); + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> intermediaryDataManager.deletePartitions(supervisorTaskId)), + DruidExceptionMatcher.invalidInput().expectMessageIs( + StringUtils.format( + "Invalid value for field [supervisorTaskId]: Value [%s] cannot start with '.'.", + supervisorTaskId + ) + ) + ); Assert.assertTrue(new File(intermediarySegmentsLocation, supervisorTaskId).exists()); Assert.assertTrue(dataFile.exists()); } @@ -186,8 +195,6 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest @Test public void testFailsWithCraftyFabricatedNamesForFind() throws IOException { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("supervisorTaskId cannot start with the '.' character."); final String supervisorTaskId = "../" + siblingLocation.getName(); final Interval interval = Intervals.of("2018/2019"); final int partitionId = 0; @@ -211,13 +218,22 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest Assert.assertTrue( new File(intermediarySegmentsLocation, supervisorTaskId + "/" + someFilePath).exists()); - final Optional foundFile1 = intermediaryDataManager.findPartitionFile( - supervisorTaskId, - someFile, - interval, - partitionId + + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> + intermediaryDataManager.findPartitionFile( + supervisorTaskId, + someFile, + interval, + partitionId + )), + DruidExceptionMatcher.invalidInput().expectMessageIs( + StringUtils.format( + "Invalid value for field [supervisorTaskId]: Value [%s] cannot start with '.'.", + supervisorTaskId + ) + ) ); - Assert.assertFalse(foundFile1.isPresent()); } private File generateSegmentDir(String fileName) throws IOException diff --git a/integration-tests/src/test/java/org/apache/druid/tests/query/ITJdbcQueryTest.java b/integration-tests/src/test/java/org/apache/druid/tests/query/ITJdbcQueryTest.java index 72e682c9fbf..8d07f1b8d60 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/query/ITJdbcQueryTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/query/ITJdbcQueryTest.java @@ -213,13 +213,15 @@ public class ITJdbcQueryTest } } - @Test(expectedExceptions = AvaticaSqlException.class, expectedExceptionsMessageRegExp = ".* Parameter at position \\[0] is not bound") + @Test(expectedExceptions = AvaticaSqlException.class, expectedExceptionsMessageRegExp = ".* No value bound for parameter \\(position \\[1]\\)") public void testJdbcPrepareStatementQueryMissingParameters() throws SQLException { for (String url : connections) { try (Connection connection = DriverManager.getConnection(url, connectionProperties); PreparedStatement statement = connection.prepareStatement(QUERY_PARAMETERIZED); ResultSet resultSet = statement.executeQuery()) { + // This won't actually run as we expect the exception to be thrown before it gets here + throw new IllegalStateException(resultSet.toString()); } } } diff --git a/processing/src/main/java/org/apache/druid/common/exception/DruidException.java b/processing/src/main/java/org/apache/druid/common/exception/DruidException.java index 638653fc5ce..42a679b6bd2 100644 --- a/processing/src/main/java/org/apache/druid/common/exception/DruidException.java +++ b/processing/src/main/java/org/apache/druid/common/exception/DruidException.java @@ -21,7 +21,11 @@ package org.apache.druid.common.exception; /** * A generic exception thrown by Druid. + * + * This class is deprecated and should not be used. {@link org.apache.druid.error.DruidException} should be used for + * any error that is intended to be delivered to the end user. */ +@Deprecated public class DruidException extends RuntimeException { public static final int HTTP_CODE_SERVER_ERROR = 500; diff --git a/processing/src/main/java/org/apache/druid/common/utils/IdUtils.java b/processing/src/main/java/org/apache/druid/common/utils/IdUtils.java index 2d3f3301016..88d4d0d413b 100644 --- a/processing/src/main/java/org/apache/druid/common/utils/IdUtils.java +++ b/processing/src/main/java/org/apache/druid/common/utils/IdUtils.java @@ -21,10 +21,9 @@ package org.apache.druid.common.utils; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.IAE; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -43,23 +42,32 @@ public class IdUtils public static String validateId(String thingToValidate, String stringToValidate) { - Preconditions.checkArgument( - !Strings.isNullOrEmpty(stringToValidate), - "%s cannot be null or empty. Please provide a %s.", thingToValidate, thingToValidate - ); - Preconditions.checkArgument( - !stringToValidate.startsWith("."), - "%s cannot start with the '.' character.", thingToValidate - ); - Preconditions.checkArgument( - !stringToValidate.contains("/"), - "%s cannot contain the '/' character.", thingToValidate - ); + if (Strings.isNullOrEmpty(stringToValidate)) { + throw InvalidInput.exception("Invalid value for field [%s]: must not be null", thingToValidate); + } + if (stringToValidate.startsWith(".")) { + throw InvalidInput.exception( + "Invalid value for field [%s]: Value [%s] cannot start with '.'.", + thingToValidate, + stringToValidate + ); + } + if (stringToValidate.contains("/")) { + throw InvalidInput.exception( + "Invalid value for field [%s]: Value [%s] cannot contain '/'.", + thingToValidate, + stringToValidate + ); + } + Matcher m = INVALIDCHARS.matcher(stringToValidate); - Preconditions.checkArgument( - !m.matches(), - "%s cannot contain whitespace character except space.", thingToValidate - ); + if (m.matches()) { + throw InvalidInput.exception( + "Invalid value for field [%s]: Value [%s] contains illegal whitespace characters. Only space is allowed.", + thingToValidate, + stringToValidate + ); + } for (int i = 0; i < stringToValidate.length(); i++) { final char c = stringToValidate.charAt(i); @@ -68,7 +76,13 @@ public class IdUtils // znode paths. The first two ranges are control characters, the second two ranges correspond to surrogate // pairs. This means that characters outside the basic multilingual plane, such as emojis, are not allowed. 😢 if (c > 0 && c < 31 || c > 127 && c < 159 || c > '\ud800' && c < '\uf8ff' || c > '\ufff0' && c < '\uffff') { - throw new IAE("%s cannot contain character #%d (at position %d).", thingToValidate, (int) c, i); + throw InvalidInput.exception( + "Invalid value for field [%s]: Value [%s] contains illegal UTF8 character [#%d] at position [%d]", + thingToValidate, + stringToValidate, + (int) c, + i + ); } } @@ -94,7 +108,12 @@ public class IdUtils return newTaskId(null, typeName, dataSource, interval); } - public static String newTaskId(@Nullable String idPrefix, String typeName, String dataSource, @Nullable Interval interval) + public static String newTaskId( + @Nullable String idPrefix, + String typeName, + String dataSource, + @Nullable Interval interval + ) { return newTaskId(idPrefix, getRandomId(), DateTimes.nowUtc(), typeName, dataSource, interval); } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java index a6ebb0a9113..b5a61f69292 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/JsonNodeReader.java @@ -56,7 +56,7 @@ import java.util.Map; *

* The input text can be: * 1. a JSON string of an object in a line or multiple lines(such as pretty-printed JSON text) - * 2. multiple JSON object strings concated by white space character(s) + * 2. multiple JSON object strings concatenated by white space character(s) *

* If an input string contains invalid JSON syntax, any valid JSON objects found prior to encountering the invalid * syntax will be successfully parsed, but parsing will not continue after the invalid syntax. diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java new file mode 100644 index 00000000000..6acedf55fdb --- /dev/null +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -0,0 +1,449 @@ +/* + * 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.druid.error; + +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.StringUtils; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * Represents an error condition exposed to the user and/or operator of Druid. Given that a DruidException is intended + * to be delivered to the end user, it should generally never be caught. DruidExceptions are generated at terminal + * points where the operation that was happening cannot make forward progress. As such, the only reason to catch a + * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using + * {@link #prependAndBuild(String, Object...)}. If code wants to catch and handle an exception instead, it should not + * be using the DruidException. + *

+ * Said another way, when a developer builds a DruidException in the code, they should be confident that the exception + * will make its way back to the user. DruidException is always the answer to "how do I generate an error message and + * deliver it to the user"? + *

+ * At the time that DruidException was introduced, this type of "show this to the user please" exception was largely + * handled by created {@link org.apache.druid.java.util.common.RE}, {@link org.apache.druid.java.util.common.IAE}, or + * {@link org.apache.druid.java.util.common.ISE} objects. It is intended that DruidException replaces all usage of + * these exceptions where the intention is to deliver a message to the user, which we believe to be the vast majority + * of usages. In cases where those exceptions are with the intention of being caught and acted upon, they should + * no change should occur. + * + * Notes about exception messages + * + * Firstly, exception messages should always be written with the notions from the style conventions covered in + * {@code dev/style-conventions.md}. Whenever possible, we should also try to provide an action to take to resolve + * the issue. + * + * Secondly, given that the DruidException requires defining a target persona, exception messages should always be + * written with that target persona in mind. Reviewers should use the targetPersona as added input to help validate + * that an exception message in meaningful. + * + * For example, at the time that this exception was introduced, there is an exception that the router throws which is + * an {@link org.apache.druid.java.util.common.ISE} with the message {@code "No default server found!"}. This + * exception is thrown when the router is unable to find a broker to forward a request to. It is completely + * meaningless to an end-user trying to run a query (what's a default server? why does it need to be found?). If we + * were to convert the exception to a DruidException and keep the same message, we should mark it as targetting the + * DEVELOPER persona as that is the only persona who should actually be able to figure out what a default server is + * and why it is important. That said, does it makes sense for an exception that means "router cannot find a broker + * to forward the query to" to only be targetting the DEVELOPER? The answer to that is no, it's something that should + * really be made meaningful to a wider group. Some options could be + * + * USER persona: Cannot find a queryable server, contact your cluster administrator to validate that all services are + * operational + * + * OPERATOR persona: Router unable to find a broker, check that brokers are up and active + * + * The user-facing message doesn't talk about any Druid-specific concepts and just tries to relay a high-level + * understanding of what happened. The admin-facing message includes Druid notions in it as it expects that an Admin + * will understand the various node types of Druid. + * + * If we think about this error more, we will realize that it's fundamentally something wrong with the cluster setup, + * which is something that we would expect an operator to be in charge of. So, we would pick the OPERATOR persona + * message, which also allows us to include more specific information about what server was not found and provide a + * more meaningful action to take (check the health of your brokers). + * + * Description of fields of DruidException + * Every error consists of: + *

    + *
  • A target persona
  • + *
  • A categorization of the error
  • + *
  • An error code
  • + *
  • An error message
  • + *
  • A context (possibly empty)
  • + *
+ *

+ *

+ * The target persona indicates who the message is written for. This is important for 2 reasons + *

    + *
  1. It identifies why the developer is creating the exception and who they believe can take action on it. + * This context allows for code reviewers and other developers to evaluate the message with the persona in mind
  2. + *
  3. It can be used as a way to control which error messages should be routed where. For example, a user-targetted + * error message should be able to be exposed directly to the user, while an operator-targetted error message should + * perhaps be routed to the operators of the system instead of the end user firing a query.
  4. + *
+ *

+ * The category indicates what kind of failure occurred. This is leveraged to align response codes (e.g. HTTP response + * codes) for similar exception messages. + *

+ * The error code is a code that indicates a grouping of error messages. There is no forced structure around whether + * a specific error code can be reused for different problems or not. That is, an error code like "general" will get + * reused in many different places as it's the basic error code used whenever a DruidException is created in-line. But, + * we might decide that a specific type of error should be identified explicitly by its error code and should only mean + * one thing, in which case that error code might only exist on a single error. + *

+ * The error message is a message written targetting the target persona. It should have values interpolated into it + * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information. + *

+ * The context is a place to add extra information about the error that is not necessarily interpolated into the + * error message. It's a way to carry extra information that might be useful to a developer, but not necessarily to + * the target persona. + * + * Notes for developers working with DruidException + *

+ * A DruidException can be built from one of 2 static methods: {@link #forPersona} or {@link #fromFailure(Failure)}. + * The only way to set a specific error code is to build a DruidException from a Failure, when built in-line using + * forPersona, it will always be an "general" error. + *

+ * Additionally, DruidException is not intended to be directly serialized. The intention is that something converts + * it into an {@link ErrorResponse} first using {@link ErrorResponse#ErrorResponse(DruidException)} and then that + * ErrorResponse is used for serialization. DruidException carries a {@link #toErrorResponse()} method because there + * are some code paths that directly serialize Exceptions and adjusting them was deemed out-of-scope for the PR that + * introduced DruidException. + */ +@NotThreadSafe +public class DruidException extends RuntimeException +{ + /** + * Starts building an "general" DruidException targetting the specific persona. + * + * @param persona the target persona of the exception message + * @return a builder that can be used to complete the creation of the DruidException + */ + public static PartialDruidExceptionBuilder forPersona(Persona persona) + { + return new PartialDruidExceptionBuilder("general", persona); + } + + /** + * Builds a DruidException using the provided Failure class. The errorCode is determined by the + * specific Failure class being used and the Failure class is responsible for setting all other + * required fields of the DruidException + * + * @param failure failure implementation to use to build the DruidException + * @return DruidException instance built from the Failure instance provided + */ + public static DruidException fromFailure(Failure failure) + { + return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode())); + } + + private final Persona targetPersona; + private final Category category; + private final String errorCode; + protected final Map context = new LinkedHashMap<>(); + + private DruidException( + Throwable cause, + final String errorCode, + Persona targetPersona, + Category category, + final String message + ) + { + this(cause, errorCode, targetPersona, category, message, false); + } + + private DruidException( + Throwable throwable, + final String errorCode, + Persona targetPersona, + Category category, + String message, + boolean deserialized + ) + { + super(message, throwable, true, !deserialized); + this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode"); + this.targetPersona = Preconditions.checkNotNull(targetPersona, "targetPersona"); + this.category = Preconditions.checkNotNull(category, "category"); + } + + public DruidException withContext(String key, Object value) + { + context.put(key, value == null ? null : value.toString()); + return this; + } + + public DruidException withContext(Map values) + { + this.context.putAll(values); + return this; + } + + public Persona getTargetPersona() + { + return targetPersona; + } + + public Category getCategory() + { + return category; + } + + public String getErrorCode() + { + return errorCode; + } + + public String getContextValue(String key) + { + return context.get(key); + } + + public Map getContext() + { + return context; + } + + public int getStatusCode() + { + return category.getExpectedStatus(); + } + + /** + * Returns this DruidException as an ErrorResponse. This method exists for compatibility with some older code + * paths that serialize out Exceptions directly using Jackson. Instead of serializing a DruidException + * directly, code should be structured to take the DruidException and build an ErrorResponse from it to be + * used to push across the wire. + *

+ * As such, this method should be deleted in some future world. Anyone wondering how to serialize and deserialize + * a DruidException should look at {@link ErrorResponse} and leverage that instead of this. + * + * @return an ErrorResponse + */ + @SuppressWarnings("unused") + @JsonValue + public ErrorResponse toErrorResponse() + { + return new ErrorResponse(this); + } + + /** + * Builds a new DruidException with a message that is the result of prepending the message passed as a parameter + * with the message already on the DruidException. + * + * @param msg Message to be prepended, can be a Java format string + * @param args Arguments to be passed to the message if it is a Java format string + * @return a new DruidException with prepended-message + */ + public DruidException prependAndBuild(String msg, Object... args) + { + return new DruidException( + this, + errorCode, + targetPersona, + category, + StringUtils.format("%s: %s", StringUtils.nonStrictFormat(msg, args), getMessage()) + ).withContext(context); + } + + /** + * The persona that the message on a DruidException is targetting + */ + public enum Persona + { + /** + * Represents the end-user, a persona who is issuing queries to the Druid Query APIs + */ + USER, + /** + * Represents an administrative user, a persona who is interacting with admin APIs and understands Druid query + * concepts without necessarily owning the infrastructure and operations of the cluster + */ + ADMIN, + /** + * Represents a persona who actively owns and operates the cluster. This persona is not assumed to understand + * Druid query concepts, but instead understand cluster operational concepts. + */ + OPERATOR, + /** + * Represents someone who has all of the context and knowledge to be actively diving into the Druid codebase. + * This persona exists as a catch-all for anything that is so deep and technically in the weeds that it is not + * possible to make a message that will make sense to a different persona. Generally speaking, there is a hope + * that only DEFENSIVE error messages will target this persona. + */ + DEVELOPER + } + + /** + * Category of error. The simplest way to describe this is that it exists as a classification of errors that + * enables us to identify the expected response code (e.g. HTTP status code) of a specific DruidException + */ + public enum Category + { + /** + * Means that the exception is being created defensively, because we want to validate something but expect that + * it should never actually be hit. Using this category is good to provide an indication to future reviewers and + * developers that the case being checked is not intended to actually be able to occur in the wild. + */ + DEFENSIVE(500), + /** + * Means that the input provided was malformed in some way. Generally speaking, it is hoped that errors of this + * category have messages written either targetting the USER or ADMIN personas as those are the general users + * of the APIs who could generate invalid inputs. + */ + INVALID_INPUT(400), + /** + * Means that the error is a problem with authorization. + */ + UNAUTHORIZED(401), + /** + * Means that some capacity limit was exceeded, this could be due to throttling or due to some system limit + */ + CAPACITY_EXCEEDED(429), + /** + * Means that the query was canceled for some reason + */ + CANCELED(500), + /** + * Indicates a server-side failure of some sort at runtime + */ + RUNTIME_FAILURE(500), + /** + * A timeout happened + */ + TIMEOUT(504), + /** + * Indicates some unsupported behavior was requested. + */ + UNSUPPORTED(501), + /** + * A catch-all for any time when we cannot come up with a meaningful categorization. This is hopefully only + * used when converting generic exceptions from frameworks and libraries that we do not control into DruidExcpetions + */ + UNCATEGORIZED(500); + + private final int expectedStatus; + + Category(int expectedStatus) + { + this.expectedStatus = expectedStatus; + } + + public int getExpectedStatus() + { + return expectedStatus; + } + } + + public static class PartialDruidExceptionBuilder + { + private String errorCode; + private Persona targetPersona; + + private PartialDruidExceptionBuilder(String errorCode, Persona targetPersona) + { + this.errorCode = errorCode; + this.targetPersona = targetPersona; + } + + public DruidExceptionBuilder ofCategory(Category category) + { + return new DruidExceptionBuilder(errorCode).forPersona(targetPersona).ofCategory(category); + } + } + + public static class DruidExceptionBuilder + { + private String errorCode; + private Persona targetPersona; + private Category category; + + private boolean deserialized = false; + + private DruidExceptionBuilder(String errorCode) + { + this.errorCode = errorCode; + } + + public DruidExceptionBuilder forPersona(Persona targetPersona) + { + this.targetPersona = targetPersona; + return this; + } + + public DruidExceptionBuilder ofCategory(Category category) + { + this.category = category; + return this; + } + + /** + * Exists for ErrorMessage to be able to indicate that the exception was deserialized and (therefore) + * should not carry any stack-trace as the stack-trace generated would be to the deserialization code rather than + * the actual error. + * + * @return the builder + */ + DruidExceptionBuilder wasDeserialized() + { + this.deserialized = true; + return this; + } + + public DruidException build(String formatMe, Object... vals) + { + return build(null, formatMe, vals); + } + + public DruidException build(Throwable cause, String formatMe, Object... vals) + { + return new DruidException( + cause, + errorCode, + targetPersona, + category, + StringUtils.nonStrictFormat(formatMe, vals), + deserialized + ); + } + } + + public abstract static class Failure + { + private final String errorCode; + + public Failure( + String errorCode + ) + { + this.errorCode = errorCode; + } + + public String getErrorCode() + { + return errorCode; + } + + protected abstract DruidException makeException(DruidExceptionBuilder bob); + } + +} diff --git a/processing/src/main/java/org/apache/druid/error/ErrorResponse.java b/processing/src/main/java/org/apache/druid/error/ErrorResponse.java new file mode 100644 index 00000000000..7b571cca271 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/error/ErrorResponse.java @@ -0,0 +1,214 @@ +/* + * 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.druid.error; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import org.apache.druid.query.QueryException; + +import javax.annotation.Nullable; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * A Response Object that represents an error to be returned over the wire. This object carries legacy bits to + * deal with compatibility issues of converging the error responses from {@link QueryException} + * with the intended going-forward error responses from {@link DruidException} + *

+ * The intent is that eventually {@link QueryException} is completely subsumed by + * {@link DruidException} in which case the legacy bits of this class can hopefully also be removed. + *

+ * The intended long-term schema of output is an object that looks like + *

+ * { + * "errorCode": `a code string`, + * "persona": USER | ADMIN | OPERATOR | DEVELOPER + * "category": DEFENSIVE | INVALID_INPUT | UNAUTHORIZED | CAPACITY_EXCEEDED | CANCELED | RUNTIME_FAILURE | TIMEOUT | UNSUPPORTED | UNCATEGORIZED + * "errorMessage": `a message for the intended audience` + * "context": `a map of extra context values that might be helpful` + * } + *

+ * In the interim, there are extra fields that also end up included so that the wire-schema can also be interpretted + * and handled by clients that are built assuming they are looking at QueryExceptions. These extra fields are + *

+ * { + * "error": `an error code from QueryException` | "druidException" + * "errorClass": `the error class, as used by QueryException` + * "host": `the host that the exception occurred on, as used by QueryException` + * } + *

+ * These 3 top-level fields are deprecated and will eventually disappear from API responses. The values can, instead, + * be pulled from the context object of an "legacyQueryException" errorCode object. The field names in the context + * object map as follows + * * "error" -> "legacyErrorCode" + * * "errorClass" -> "errorClass" + * * "host" -> "host" + */ +public class ErrorResponse +{ + @JsonCreator + public static ErrorResponse fromMap(Map map) + { + final DruidException.Failure failure; + + final Object legacyErrorType = map.get("error"); + if (!"druidException".equals(legacyErrorType)) { + // The non "druidException" errorCode field means that we are deserializing a legacy QueryException object rather + // than deserializing a DruidException. So, we make a QueryException, map it to a DruidException and build + // our response from that DruidException. This allows all code after us to only consider DruidException + // and helps aid the removal of QueryException. + failure = new QueryExceptionCompat( + new QueryException( + nullOrString(map.get("error")), + nullOrString(map.get("errorMessage")), + nullOrString(map.get("errorClass")), + nullOrString(map.get("host")) + ) + ); + } else { + failure = new DruidException.Failure(stringOrFailure(map, "errorCode")) + { + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + final DruidException retVal = bob.forPersona(DruidException.Persona.valueOf(stringOrFailure(map, "persona"))) + .ofCategory(DruidException.Category.valueOf(stringOrFailure( + map, + "category" + ))) + .build(stringOrFailure(map, "errorMessage")); + + final Object context = map.get("context"); + if (context instanceof Map) { + //noinspection unchecked + retVal.withContext((Map) context); + } + + return retVal; + } + }; + } + return new ErrorResponse(DruidException.fromFailure(new DeserializedFailure(failure))); + } + + private final DruidException underlyingException; + + public ErrorResponse(DruidException underlyingException) + { + this.underlyingException = underlyingException; + } + + @JsonValue + public Map getAsMap() + { + final LinkedHashMap retVal = new LinkedHashMap<>(); + + // This if statement is a compatibility layer to help bridge the time while we are introducing the DruidException. + // In a future release, QueryException should be completely eliminated, at which point we should also be + // able to eliminate this compatibility layer. + if (QueryExceptionCompat.ERROR_CODE.equals(underlyingException.getErrorCode())) { + retVal.put("error", underlyingException.getContextValue("legacyErrorCode")); + retVal.put("errorClass", underlyingException.getContextValue("errorClass")); + retVal.put("host", underlyingException.getContextValue("host")); + } else { + retVal.put("error", "druidException"); + } + + retVal.put("errorCode", underlyingException.getErrorCode()); + retVal.put("persona", underlyingException.getTargetPersona().toString()); + retVal.put("category", underlyingException.getCategory().toString()); + retVal.put("errorMessage", underlyingException.getMessage()); + retVal.put("context", underlyingException.getContext()); + + return retVal; + } + + public DruidException getUnderlyingException() + { + return underlyingException; + } + + @Nullable + private static String nullOrString(Object o) + { + return o == null ? null : o.toString(); + } + + private static String stringOrFailure(Map map, String key) + { + final Object o = map.get(key); + if (o instanceof String) { + return (String) o; + } + + final DruidException problem = DruidException + .forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.DEFENSIVE) + .build("Got an error response that had a non-String value [%s] for key [%s]", o, key); + + for (Map.Entry entry : map.entrySet()) { + final Object value = entry.getValue(); + if (value != null) { + problem.withContext(entry.getKey(), value.toString()); + } + } + + throw problem; + } + + private static class DeserializedFailure extends DruidException.Failure + { + private final DruidException.Failure delegate; + + public DeserializedFailure( + DruidException.Failure delegate + ) + { + super(delegate.getErrorCode()); + this.delegate = delegate; + } + + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + // By setting wasDeserialized, we get the initial exception built with no stack-trace, we then create a new + // exception with the exact same values that will contain our current stack-trace and to be relevant inside + // of the current process. It's a little bit of a weird dance to create a new exception with the same stuff, + // it might be nice to have a DelegatingDruidException or something like that which looks like a DruidException + // but just delegates everything. That's something that can be explored another day though. + bob.wasDeserialized(); + final DruidException cause = delegate.makeException(bob); + + return DruidException.fromFailure( + new DruidException.Failure(cause.getErrorCode()) + { + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + return bob.forPersona(cause.getTargetPersona()) + .ofCategory(cause.getCategory()) + .build(cause, cause.getMessage()) + .withContext(cause.getContext()); + } + } + ); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/error/InvalidInput.java b/processing/src/main/java/org/apache/druid/error/InvalidInput.java new file mode 100644 index 00000000000..ce50d4db376 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/error/InvalidInput.java @@ -0,0 +1,63 @@ +/* + * 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.druid.error; + +public class InvalidInput extends DruidException.Failure +{ + public static DruidException exception(String msg, Object... args) + { + return exception(null, msg, args); + } + + public static DruidException exception(Throwable t, String msg, Object... args) + { + return DruidException.fromFailure(new InvalidInput(t, msg, args)); + } + + private final Throwable t; + private final String msg; + private final Object[] args; + + public InvalidInput( + Throwable t, + String msg, + Object... args + ) + { + super("invalidInput"); + this.t = t; + this.msg = msg; + this.args = args; + } + + + @Override + public DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + bob = bob.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT); + + if (t == null) { + return bob.build(msg, args); + } else { + return bob.build(t, msg, args); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/error/InvalidSqlInput.java b/processing/src/main/java/org/apache/druid/error/InvalidSqlInput.java new file mode 100644 index 00000000000..17a392962f9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/error/InvalidSqlInput.java @@ -0,0 +1,50 @@ +/* + * 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.druid.error; + +public class InvalidSqlInput extends InvalidInput +{ + public static DruidException exception(String msg, Object... args) + { + return exception(null, msg, args); + } + + public static DruidException exception(Throwable t, String msg, Object... args) + { + return DruidException.fromFailure(new InvalidSqlInput(t, msg, args)); + } + + public InvalidSqlInput( + Throwable t, + String msg, + Object... args + ) + { + super(t, msg, args); + } + + @Override + public DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + final DruidException retVal = super.makeException(bob); + retVal.withContext("sourceType", "sql"); + return retVal; + } +} diff --git a/processing/src/main/java/org/apache/druid/error/QueryExceptionCompat.java b/processing/src/main/java/org/apache/druid/error/QueryExceptionCompat.java new file mode 100644 index 00000000000..12e4905efae --- /dev/null +++ b/processing/src/main/java/org/apache/druid/error/QueryExceptionCompat.java @@ -0,0 +1,79 @@ +/* + * 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.druid.error; + +import org.apache.druid.query.QueryException; + +/** + * A {@link DruidException.Failure} that serves to cover conversions from {@link QueryException}. + * + * When/if QueryException is completely eliminated from the code base, this compat layer should also be able to + * be removed. Additionally, it is the hope that nobody should actually be interacting with this class as it should + * be an implementation detail of {@link DruidException} and not really seen outside of that. + */ +public class QueryExceptionCompat extends DruidException.Failure +{ + public static final String ERROR_CODE = "legacyQueryException"; + + private final QueryException exception; + + public QueryExceptionCompat( + QueryException exception + ) + { + super(ERROR_CODE); + this.exception = exception; + } + + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + return bob.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(convertFailType(exception.getFailType())) + .build(exception.getMessage()) + .withContext("host", exception.getHost()) + .withContext("errorClass", exception.getErrorClass()) + .withContext("legacyErrorCode", exception.getErrorCode()); + } + + private DruidException.Category convertFailType(QueryException.FailType failType) + { + switch (failType) { + case USER_ERROR: + return DruidException.Category.INVALID_INPUT; + case UNAUTHORIZED: + return DruidException.Category.UNAUTHORIZED; + case CAPACITY_EXCEEDED: + return DruidException.Category.CAPACITY_EXCEEDED; + case QUERY_RUNTIME_FAILURE: + return DruidException.Category.RUNTIME_FAILURE; + case CANCELED: + return DruidException.Category.CANCELED; + case UNKNOWN: + return DruidException.Category.UNCATEGORIZED; + case UNSUPPORTED: + return DruidException.Category.UNSUPPORTED; + case TIMEOUT: + return DruidException.Category.TIMEOUT; + default: + return DruidException.Category.UNCATEGORIZED; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/java/util/common/logger/Logger.java b/processing/src/main/java/org/apache/druid/java/util/common/logger/Logger.java index 3872c2cf768..1639bf5378f 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/logger/Logger.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/logger/Logger.java @@ -33,6 +33,37 @@ import java.util.Iterator; import java.util.function.BiConsumer; import java.util.stream.Stream; +/** + * A Logger for usage inside of Druid. Provides a layer that allows for simple changes to the logging framework + * with minimal changes to the Druid code. + * + * Log levels are used as an indication of urgency around the behavior that is being logged. The intended generic + * rubric for when to use the different logging levels is as follows. + * + * DEBUG: something that a developer wants to look at while actively debugging, but should not be included by default. + * + * INFO: a message that is useful to have when trying to retro-actively understand what happened in a running system. + * There is often a fine line between INFO and DEBUG. We want information from INFO logs but do not want to spam log + * files either. One rubric to use to help determine if something should be INFO or DEBUG is how often we expect the + * line to be logged. If there is clarity that it will happen in a controlled manner such that it does not spam the + * logs, then INFO is fine. Additionally, it can be okay to log at INFO level even if there is a risk of spamming the + * log file in the case that the log line only happens in specific "error-oriented" situations, this is because such + * error-oriented situations are more likely to necessitate reading and understanding the logs to eliminate the error. + * Additionally, it is perfectly acceptable and reasonable to log an exception at INFO level. + * + * WARN: a message that indicates something bad has happened in the system that a human should potentially investigate. + * While it is bad and deserves investigation, it is of a nature that it should be able to wait until the next + * "business day" for investigation instead of needing immediate attention. + * + * ERROR: a message that indicates that something bad has happened such that a human operator should take immediate + * intervention to triage and resolve the issue as it runs a risk to the smooth operations of the system. Logs at + * the ERROR level should generally be severe enough to warrant paging someone in the middle of the night. + * + * Even though this is the intended rubric, it is very difficult to ensure that, e.g. all ERROR log lines are pageable + * offenses. As such, it is questionable whether an operator should actually ALWAYS page on every ERROR log line, + * but as a directional target of when and how to log things, the above rubric should be used to evaluate if a log + * line is at the correct level. + */ public class Logger { @VisibleForTesting diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedPathFinder.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedPathFinder.java index b275199bb0e..5d0596a6cc8 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedPathFinder.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedPathFinder.java @@ -19,7 +19,8 @@ package org.apache.druid.segment.nested; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nullable; import java.util.ArrayList; @@ -74,7 +75,7 @@ public class NestedPathFinder List parts = new ArrayList<>(); if (!path.startsWith(JSON_PATH_ROOT)) { - badFormatJsonPath(path, "must start with '$'"); + badFormatJsonPath(path, "it must start with '$'"); } if (path.length() == 1) { @@ -97,7 +98,7 @@ public class NestedPathFinder partMark = i + 1; } else if (current == '[' && arrayMark < 0 && quoteMark < 0) { if (dotMark == (i - 1) && dotMark != 0) { - badFormatJsonPath(path, "invalid position " + i + " for '[', must not follow '.' or must be contained with '"); + badFormatJsonPath(path, "found '[' at invalid position [%s], must not follow '.' or must be contained with '", i); } if (dotMark >= 0 && i > 1) { parts.add(new NestedPathField(getPathSubstring(path, partMark, i))); @@ -115,13 +116,13 @@ public class NestedPathFinder partMark = i + 1; } catch (NumberFormatException ignored) { - badFormatJsonPath(path, "expected number for array specifier got " + maybeNumber + " instead. Use ' if this value was meant to be a field name"); + badFormatJsonPath(path, "array specifier [%s] should be a number, it was not. Use ' if this value was meant to be a field name", maybeNumber); } } else if (dotMark == -1 && arrayMark == -1) { badFormatJsonPath(path, "path parts must be separated with '.'"); } else if (current == '\'' && quoteMark < 0) { if (arrayMark != i - 1) { - badFormatJsonPath(path, "' must be immediately after '['"); + badFormatJsonPath(path, "single-quote (') must be immediately after '['"); } quoteMark = i; partMark = i + 1; @@ -130,7 +131,7 @@ public class NestedPathFinder if (arrayMark >= 0) { continue; } - badFormatJsonPath(path, "closing ' must immediately precede ']'"); + badFormatJsonPath(path, "closing single-quote (') must immediately precede ']'"); } parts.add(new NestedPathField(getPathSubstring(path, partMark, i))); @@ -147,7 +148,7 @@ public class NestedPathFinder // add the last element, this should never be an array because they close themselves if (partMark < path.length()) { if (quoteMark != -1) { - badFormatJsonPath(path, "unterminated '"); + badFormatJsonPath(path, "unterminated single-quote (')"); } if (arrayMark != -1) { badFormatJsonPath(path, "unterminated '['"); @@ -195,7 +196,7 @@ public class NestedPathFinder List parts = new ArrayList<>(); if (path.charAt(0) != '.') { - badFormat(path, "must start with '.'"); + badFormat(path, "it must start with '.'"); } int partMark = -1; // position to start the next substring to build the path part @@ -217,13 +218,13 @@ public class NestedPathFinder parts.add(new NestedPathField(getPathSubstring(path, partMark, i))); dotMark = -1; } else { - badFormat(path, "invalid position " + i + " for '?'"); + badFormat(path, "found '?' at invalid position [%s]", i); } } partMark = i + 1; } else if (current == '[' && arrayMark < 0 && quoteMark < 0) { if (dotMark == (i - 1) && dotMark != 0) { - badFormat(path, "invalid position " + i + " for '[', must not follow '.' or must be contained with '\"'"); + badFormat(path, "found '[' at invalid position [%s], must not follow '.' or must be contained with '\"'", i); } if (dotMark >= 0 && i > 1) { parts.add(new NestedPathField(getPathSubstring(path, partMark, i))); @@ -241,16 +242,16 @@ public class NestedPathFinder partMark = i + 1; } catch (NumberFormatException ignored) { - badFormat(path, "expected number for array specifier got " + maybeNumber + " instead. Use \"\" if this value was meant to be a field name"); + badFormat(path, "array specifier [%s] should be a number, it was not. Use \"\" if this value was meant to be a field name", maybeNumber); } } else if (dotMark == -1 && arrayMark == -1) { badFormat(path, "path parts must be separated with '.'"); } else if (current == '"' && quoteMark < 0) { if (partMark != i) { - badFormat(path, "invalid position " + i + " for '\"', must immediately follow '.' or '['"); + badFormat(path, "found '\"' at invalid position [%s], it must immediately follow '.' or '['", i); } if (arrayMark > 0 && arrayMark != i - 1) { - badFormat(path, "'\"' within '[' must be immediately after"); + badFormat(path, "'\"' within '[', must be immediately after"); } quoteMark = i; partMark = i + 1; @@ -295,14 +296,14 @@ public class NestedPathFinder return path.substring(start, end); } - private static void badFormat(String path, String message) + private static void badFormat(String path, String message, Object... args) { - throw new IAE("Bad format, '%s' is not a valid 'jq' path: %s", path, message); + throw InvalidInput.exception("jq path [%s] is invalid, %s", path, StringUtils.format(message, args)); } - private static void badFormatJsonPath(String path, String message) + private static void badFormatJsonPath(String path, String message, Object... args) { - throw new IAE("Bad format, '%s' is not a valid JSONPath path: %s", path, message); + throw InvalidInput.exception("JSONPath [%s] is invalid, %s", path, StringUtils.format(message, args)); } /** diff --git a/processing/src/test/java/org/apache/druid/common/utils/IdUtilsTest.java b/processing/src/test/java/org/apache/druid/common/utils/IdUtilsTest.java index 806c74e6161..b61ef5df4cd 100644 --- a/processing/src/test/java/org/apache/druid/common/utils/IdUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/common/utils/IdUtilsTest.java @@ -19,21 +19,17 @@ package org.apache.druid.common.utils; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.junit.Assert; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; public class IdUtilsTest { private static final String THINGO = "thingToValidate"; public static final String VALID_ID_CHARS = "alpha123..*~!@#&%^&*()-+ Россия\\ 한국 中国!"; - @Rule - public ExpectedException expectedException = ExpectedException.none(); - @Test public void testValidIdName() { @@ -43,89 +39,89 @@ public class IdUtilsTest @Test public void testInvalidNull() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot be null or empty. Please provide a thingToValidate."); - IdUtils.validateId(THINGO, null); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: must not be null" + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, null)); } @Test public void testInvalidEmpty() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot be null or empty. Please provide a thingToValidate."); - IdUtils.validateId(THINGO, ""); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: must not be null" + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "")); } @Test public void testInvalidSlashes() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot contain the '/' character."); - IdUtils.validateId(THINGO, "/paths/are/bad/since/we/make/files/from/stuff"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: Value [/paths/are/bad/since/we/make/files/from/stuff] cannot contain '/'." + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "/paths/are/bad/since/we/make/files/from/stuff")); } @Test public void testInvalidLeadingDot() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot start with the '.' character."); - IdUtils.validateId(THINGO, "./nice/try"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: Value [./nice/try] cannot start with '.'." + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "./nice/try")); } @Test public void testInvalidSpacesRegexTabs() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot contain whitespace character except space."); - IdUtils.validateId(THINGO, "spaces\tare\tbetter\tthan\ttabs\twhich\tare\tillegal"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: Value [spaces\tare\tbetter\tthan\ttabs\twhich\tare\tillegal] contains illegal whitespace characters. Only space is allowed." + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "spaces\tare\tbetter\tthan\ttabs\twhich\tare\tillegal")); } @Test public void testInvalidSpacesRegexNewline() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot contain whitespace character except space."); - IdUtils.validateId(THINGO, "new\nline"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: Value [new\nline] contains illegal whitespace characters. Only space is allowed." + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "new\nline")); } @Test public void testInvalidSpacesRegexCarriageReturn() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot contain whitespace character except space."); - IdUtils.validateId(THINGO, "does\rexist\rby\ritself"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: Value [does\rexist\rby\ritself] contains illegal whitespace characters. Only space is allowed." + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "does\rexist\rby\ritself")); } @Test public void testInvalidSpacesRegexLineTabulation() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot contain whitespace character except space."); - IdUtils.validateId(THINGO, "what\u000Bis line tabulation"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: Value [what\u000Bis line tabulation] contains illegal whitespace characters. Only space is allowed." + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "what\u000Bis line tabulation")); } @Test public void testInvalidSpacesRegexFormFeed() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot contain whitespace character except space."); - IdUtils.validateId(THINGO, "form\u000cfeed?"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: Value [form\ffeed?] contains illegal whitespace characters. Only space is allowed." + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "form\u000cfeed?")); } @Test public void testInvalidUnprintableChars() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot contain character #129 (at position 4)."); - IdUtils.validateId(THINGO, "form\u0081feed?"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: Value [form\u0081feed?] contains illegal UTF8 character [#129] at position [4]" + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "form\u0081feed?")); } @Test public void testInvalidEmojis() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("thingToValidate cannot contain character #55357 (at position 4)."); - IdUtils.validateId(THINGO, "form💯feed?"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [thingToValidate]: Value [form\uD83D\uDCAFfeed?] contains illegal UTF8 character [#55357] at position [4]" + ).assertThrowsAndMatches(() -> IdUtils.validateId(THINGO, "form💯feed?")); } @Test diff --git a/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java b/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java new file mode 100644 index 00000000000..d3d4e057c5e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java @@ -0,0 +1,126 @@ +/* + * 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.druid.error; + +import org.apache.druid.matchers.DruidMatchers; +import org.hamcrest.Description; +import org.hamcrest.DiagnosingMatcher; +import org.hamcrest.Matcher; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.hamcrest.core.AllOf; + +import java.util.ArrayList; + +public class DruidExceptionMatcher extends DiagnosingMatcher +{ + public static DruidExceptionMatcher invalidInput() + { + return new DruidExceptionMatcher( + DruidException.Persona.USER, + DruidException.Category.INVALID_INPUT, + "invalidInput" + ); + } + + public static DruidExceptionMatcher invalidSqlInput() + { + return invalidInput().expectContext("sourceType", "sql"); + } + + private final AllOf delegate; + private final ArrayList> matcherList; + + public DruidExceptionMatcher( + DruidException.Persona targetPersona, + DruidException.Category category, + String errorCode + ) + { + matcherList = new ArrayList<>(); + matcherList.add(DruidMatchers.fn("targetPersona", DruidException::getTargetPersona, Matchers.is(targetPersona))); + matcherList.add(DruidMatchers.fn("category", DruidException::getCategory, Matchers.is(category))); + matcherList.add(DruidMatchers.fn("errorCode", DruidException::getErrorCode, Matchers.is(errorCode))); + + delegate = new AllOf<>(matcherList); + } + + public DruidExceptionMatcher expectContext(String key, String value) + { + matcherList.add(DruidMatchers.fn("context", DruidException::getContext, Matchers.hasEntry(key, value))); + return this; + } + + public DruidExceptionMatcher expectMessageIs(String s) + { + return expectMessage(Matchers.equalTo(s)); + } + + public DruidExceptionMatcher expectMessageContains(String contains) + { + return expectMessage(Matchers.containsString(contains)); + } + + public DruidExceptionMatcher expectMessage(Matcher messageMatcher) + { + matcherList.add(DruidMatchers.fn("message", DruidException::getMessage, messageMatcher)); + return this; + } + + public DruidExceptionMatcher expectException(Matcher causeMatcher) + { + matcherList.add(DruidMatchers.fn("cause", DruidException::getCause, causeMatcher)); + return this; + } + + @Override + protected boolean matches(Object item, Description mismatchDescription) + { + return delegate.matches(item, mismatchDescription); + } + + @Override + public void describeTo(Description description) + { + delegate.describeTo(description); + } + + public void assertThrowsAndMatches(ThrowingSupplier fn) + { + boolean thrown = false; + try { + fn.get(); + } + catch (Throwable e) { + if (e instanceof DruidException) { + MatcherAssert.assertThat(e, this); + thrown = true; + } else { + throw new RuntimeException(e); + } + } + MatcherAssert.assertThat(thrown, Matchers.is(true)); + } + + public interface ThrowingSupplier + { + void get(); + } +} diff --git a/processing/src/test/java/org/apache/druid/error/ErrorResponseTest.java b/processing/src/test/java/org/apache/druid/error/ErrorResponseTest.java new file mode 100644 index 00000000000..2ddd39aa7da --- /dev/null +++ b/processing/src/test/java/org/apache/druid/error/ErrorResponseTest.java @@ -0,0 +1,110 @@ +/* + * 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.druid.error; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.matchers.DruidMatchers; +import org.apache.druid.query.QueryTimeoutException; +import org.hamcrest.Matcher; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; +import org.junit.Test; + +import java.util.Map; + +public class ErrorResponseTest +{ + @Test + public void testSanity() + { + ErrorResponse response = new ErrorResponse(InvalidSqlInput.exception("bad sql!")); + + final Map asMap = response.getAsMap(); + MatcherAssert.assertThat( + asMap, + DruidMatchers.mapMatcher( + "error", "druidException", + "errorCode", "invalidInput", + "persona", "USER", + "category", "INVALID_INPUT", + "errorMessage", "bad sql!", + "context", ImmutableMap.of("sourceType", "sql") + ) + ); + + ErrorResponse recomposed = ErrorResponse.fromMap(asMap); + + MatcherAssert.assertThat( + recomposed.getUnderlyingException(), + DruidExceptionMatcher.invalidSqlInput().expectMessageIs("bad sql!") + ); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Test + public void testQueryExceptionCompat() + { + ErrorResponse response = new ErrorResponse( + DruidException.fromFailure(new QueryExceptionCompat(new QueryTimeoutException())) + ); + + final Map asMap = response.getAsMap(); + MatcherAssert.assertThat( + asMap, + DruidMatchers.mapMatcher( + "error", + "Query timeout", + + "errorCode", + "legacyQueryException", + + "persona", + "OPERATOR", + + "category", + "TIMEOUT", + + "errorMessage", + "Query did not complete within configured timeout period. You can increase query timeout or tune the performance of query." + ) + ); + MatcherAssert.assertThat( + asMap, + (Matcher) Matchers.hasEntry( + Matchers.is("context"), + Matchers.allOf( + DruidMatchers.mapMatcher( + "errorClass", "org.apache.druid.query.QueryTimeoutException", + "legacyErrorCode", "Query timeout" + ), + Matchers.hasKey("host") + ) + ) + ); + + ErrorResponse recomposed = ErrorResponse.fromMap(asMap); + + MatcherAssert.assertThat( + recomposed.getUnderlyingException(), + new DruidExceptionMatcher(DruidException.Persona.OPERATOR, DruidException.Category.TIMEOUT, "legacyQueryException") + .expectMessageIs("Query did not complete within configured timeout period. You can increase query timeout or tune the performance of query.") + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/matchers/DruidMatchers.java b/processing/src/test/java/org/apache/druid/matchers/DruidMatchers.java new file mode 100644 index 00000000000..de9b10bbefc --- /dev/null +++ b/processing/src/test/java/org/apache/druid/matchers/DruidMatchers.java @@ -0,0 +1,49 @@ +/* + * 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.druid.matchers; + +import org.apache.druid.java.util.common.IAE; +import org.hamcrest.Matcher; +import org.hamcrest.Matchers; + +import java.util.ArrayList; +import java.util.Map; +import java.util.function.Function; + +public class DruidMatchers +{ + public static LambdaMatcher fn(String name, Function fn, Matcher matcher) + { + return new LambdaMatcher<>(name + ": ", fn, matcher); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + public static Matcher> mapMatcher(Object... keysAndValues) + { + if (keysAndValues.length % 2 == 1) { + throw new IAE("keysAndValues should be pairs, but had an odd length [%s]", keysAndValues.length); + } + ArrayList>> entryMatchers = new ArrayList<>(); + for (int i = 0; i < keysAndValues.length; i += 2) { + entryMatchers.add(Matchers.hasEntry((K) keysAndValues[i], (V) keysAndValues[i + 1])); + } + return Matchers.allOf((Iterable) entryMatchers); + } +} diff --git a/processing/src/test/java/org/apache/druid/matchers/LambdaMatcher.java b/processing/src/test/java/org/apache/druid/matchers/LambdaMatcher.java new file mode 100644 index 00000000000..3eb50466ee1 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/matchers/LambdaMatcher.java @@ -0,0 +1,62 @@ +/* + * 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.druid.matchers; + +import org.hamcrest.Description; +import org.hamcrest.DiagnosingMatcher; +import org.hamcrest.Matcher; + +import java.util.function.Function; + +public class LambdaMatcher extends DiagnosingMatcher +{ + private final String name; + private final Function fn; + private final Matcher matcher; + + public LambdaMatcher( + String name, + Function fn, + Matcher matcher + ) + { + this.name = name; + this.fn = fn; + this.matcher = matcher; + } + + @Override + protected boolean matches(Object item, Description mismatchDescription) + { + final S result = fn.apply((T) item); + if (!matcher.matches(result)) { + matcher.describeMismatch(result, mismatchDescription); + return false; + } + return true; + } + + @Override + public void describeTo(Description description) + { + description.appendText(name); + matcher.describeTo(description); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedPathFinderTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedPathFinderTest.java index 161a752ce1e..b22131833c6 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedPathFinderTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedPathFinderTest.java @@ -21,11 +21,9 @@ package org.apache.druid.segment.nested; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.error.DruidExceptionMatcher; import org.junit.Assert; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.util.List; import java.util.Map; @@ -41,9 +39,6 @@ public class NestedPathFinderTest "[also_sneaky]", ImmutableList.of(ImmutableMap.of("a", "x"), ImmutableMap.of("b", "y", "c", "z")) ); - @Rule - public ExpectedException expectedException = ExpectedException.none(); - @Test public void testParseJqPath() { @@ -188,7 +183,10 @@ public class NestedPathFinderTest Assert.assertEquals("f?o.o", pathParts.get(2).getPartIdentifier()); Assert.assertTrue(pathParts.get(3) instanceof NestedPathField); Assert.assertEquals(".b?.a.r.", pathParts.get(3).getPartIdentifier()); - Assert.assertEquals(".\"x.y.z]?[\\\"]][]\".\"13234.12[]][23\".\"f?o.o\".\".b?.a.r.\"", NestedPathFinder.toNormalizedJqPath(pathParts)); + Assert.assertEquals( + ".\"x.y.z]?[\\\"]][]\".\"13234.12[]][23\".\"f?o.o\".\".b?.a.r.\"", + NestedPathFinder.toNormalizedJqPath(pathParts) + ); } @Test @@ -334,85 +332,84 @@ public class NestedPathFinderTest @Test public void testBadFormatMustStartWithDot() { - expectedException.expect(IAE.class); - expectedException.expectMessage("Bad format, 'x.y' is not a valid 'jq' path: must start with '.'"); - NestedPathFinder.parseJqPath("x.y"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [x.y] is invalid, it must start with '.'" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath("x.y")); } @Test public void testBadFormatNoDot() { - expectedException.expect(IAE.class); - expectedException.expectMessage(".\"x\"\"y\"' is not a valid 'jq' path: path parts must be separated with '.'"); - NestedPathFinder.parseJqPath(".\"x\"\"y\""); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [.\"x\"\"y\"] is invalid, path parts must be separated with '.'" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath(".\"x\"\"y\"")); } @Test public void testBadFormatWithDot2() { - expectedException.expect(IAE.class); - expectedException.expectMessage("Bad format, '..\"x\"' is not a valid 'jq' path: path parts separated by '.' must not be empty"); - NestedPathFinder.parseJqPath("..\"x\""); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [..\"x\"] is invalid, path parts separated by '.' must not be empty" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath("..\"x\"")); } @Test public void testBadFormatWithDot3() { - expectedException.expect(IAE.class); - expectedException.expectMessage("Bad format, '.x.[1]' is not a valid 'jq' path: invalid position 3 for '[', must not follow '.' or must be contained with '\"'"); - NestedPathFinder.parseJqPath(".x.[1]"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [.x.[1]] is invalid, found '[' at invalid position [3], must not follow '.' or must be contained with '\"'" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath(".x.[1]")); } @Test public void testBadFormatWithDot4() { - expectedException.expect(IAE.class); - expectedException.expectMessage("Bad format, '.x[1].[2]' is not a valid 'jq' path: invalid position 6 for '[', must not follow '.' or must be contained with '\"'"); - NestedPathFinder.parseJqPath(".x[1].[2]"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [.x[1].[2]] is invalid, found '[' at invalid position [6], must not follow '.' or must be contained with '\"'" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath(".x[1].[2]")); } @Test public void testBadFormatNotANumber() { - expectedException.expect(IAE.class); - expectedException.expectMessage("Bad format, '.x[.1]' is not a valid 'jq' path: expected number for array specifier got .1 instead. Use \"\" if this value was meant to be a field name"); - NestedPathFinder.parseJqPath(".x[.1]"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [.x[.1]] is invalid, array specifier [.1] should be a number, it was not. Use \"\" if this value was meant to be a field name" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath(".x[.1]")); } @Test public void testBadFormatUnclosedArray() { - expectedException.expect(IAE.class); - expectedException.expectMessage("Bad format, '.x[1' is not a valid 'jq' path: unterminated '['"); - NestedPathFinder.parseJqPath(".x[1"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [.x[1] is invalid, unterminated '['" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath(".x[1")); } @Test public void testBadFormatUnclosedArray2() { - expectedException.expect(IAE.class); - expectedException.expectMessage("Bad format, '.x[\"1\"' is not a valid 'jq' path: unterminated '['"); - NestedPathFinder.parseJqPath(".x[\"1\""); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [.x[\"1\"] is invalid, unterminated '['" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath(".x[\"1\"")); } @Test public void testBadFormatUnclosedQuote() { - expectedException.expect(IAE.class); - expectedException.expectMessage("Bad format, '.x.\"1' is not a valid 'jq' path: unterminated '\"'"); - NestedPathFinder.parseJqPath(".x.\"1"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [.x.\"1] is invalid, unterminated '\"'" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath(".x.\"1")); } @Test public void testBadFormatUnclosedQuote2() { - expectedException.expect(IAE.class); - expectedException.expectMessage("Bad format, '.x[\"1]' is not a valid 'jq' path: unterminated '\"'"); - NestedPathFinder.parseJqPath(".x[\"1]"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "jq path [.x[\"1]] is invalid, unterminated '\"'" + ).assertThrowsAndMatches(() -> NestedPathFinder.parseJqPath(".x[\"1]")); } - @Test public void testPathSplitter() { diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index f4a7ab3edb7..2db205ca0be 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -33,12 +33,10 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.inject.Inject; -import org.apache.druid.client.DirectDruidClient; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Self; import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.BadJsonQueryException; import org.apache.druid.query.Query; @@ -46,7 +44,6 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryException; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.QueryToolChest; -import org.apache.druid.query.TruncatedResponseContextException; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.context.ResponseContext.Keys; import org.apache.druid.server.metrics.QueryCountStatsProvider; @@ -268,48 +265,6 @@ public class QueryResource implements QueryCountStatsProvider void incrementTimedOut(); } - public static void attachResponseContextToHttpResponse( - String queryId, - ResponseContext responseContext, - Response.ResponseBuilder responseBuilder, - ObjectMapper jsonMapper, ResponseContextConfig responseContextConfig, DruidNode selfNode - ) throws JsonProcessingException - { - transferEntityTag(responseContext, responseBuilder); - - DirectDruidClient.removeMagicResponseContextFields(responseContext); - - // Limit the response-context header, see https://github.com/apache/druid/issues/2331 - // Note that Response.ResponseBuilder.header(String key,Object value).build() calls value.toString() - // and encodes the string using ASCII, so 1 char is = 1 byte - final ResponseContext.SerializationResult serializationResult = responseContext.serializeWith( - jsonMapper, - responseContextConfig.getMaxResponseContextHeaderSize() - ); - - if (serializationResult.isTruncated()) { - final String logToPrint = StringUtils.format( - "Response Context truncated for id [%s]. Full context is [%s].", - queryId, - serializationResult.getFullResult() - ); - if (responseContextConfig.shouldFailOnTruncatedResponseContext()) { - log.error(logToPrint); - throw new QueryInterruptedException( - new TruncatedResponseContextException( - "Serialized response context exceeds the max size[%s]", - responseContextConfig.getMaxResponseContextHeaderSize() - ), - selfNode.getHostAndPortToUse() - ); - } else { - log.warn(logToPrint); - } - } - - responseBuilder.header(HEADER_RESPONSE_CONTEXT, serializationResult.getResult()); - } - private Query readQuery( final HttpServletRequest req, final InputStream in, diff --git a/server/src/main/java/org/apache/druid/server/QueryResultPusher.java b/server/src/main/java/org/apache/druid/server/QueryResultPusher.java index ad268a78ef0..074beb545b4 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResultPusher.java +++ b/server/src/main/java/org/apache/druid/server/QueryResultPusher.java @@ -23,6 +23,9 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.io.CountingOutputStream; import org.apache.druid.client.DirectDruidClient; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.ErrorResponse; +import org.apache.druid.error.QueryExceptionCompat; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -33,6 +36,7 @@ import org.apache.druid.query.QueryException; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.TruncatedResponseContextException; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.ForbiddenException; import javax.annotation.Nullable; @@ -42,7 +46,6 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import javax.ws.rs.core.StreamingOutput; import java.io.Closeable; import java.io.IOException; import java.io.OutputStream; @@ -61,9 +64,9 @@ public abstract class QueryResultPusher private final MediaType contentType; private final Map extraHeaders; - private StreamingHttpResponseAccumulator accumulator = null; - private AsyncContext asyncContext = null; - private HttpServletResponse response = null; + private StreamingHttpResponseAccumulator accumulator; + private AsyncContext asyncContext; + private HttpServletResponse response; public QueryResultPusher( HttpServletRequest request, @@ -149,6 +152,14 @@ public abstract class QueryResultPusher accumulator.close(); resultsWriter.recordSuccess(accumulator.getNumBytesSent()); } + catch (DruidException e) { + // Less than ideal. But, if we return the result as JSON, this is + // the only way for the security filter to know that, yes, it is OK + // to show the user this error even if we didn't get to the step where + // we did a security check. + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + return handleDruidException(resultsWriter, e); + } catch (QueryException e) { return handleQueryException(resultsWriter, e); } @@ -159,7 +170,7 @@ public abstract class QueryResultPusher // returning results before a ForbiddenException gets thrown, that means that we've already leaked stuff // that should not have been leaked. I.e. it means, we haven't validated the authorization early enough. if (response != null && response.isCommitted()) { - log.error(re, "Got a forbidden exception for query[%s] after the response was already committed.", queryId); + log.error(re, "Got a forbidden exception for query [%s] after the response was already committed.", queryId); } throw re; } @@ -174,17 +185,17 @@ public abstract class QueryResultPusher accumulator.close(); } catch (IOException e) { - log.warn(e, "Suppressing exception closing accumulator for query[%s]", queryId); + log.warn(e, "Suppressing exception closing accumulator for query [%s]", queryId); } } if (resultsWriter == null) { - log.warn("resultsWriter was null for query[%s], work was maybe done in start() that shouldn't be.", queryId); + log.warn("resultsWriter was null for query [%s], work was maybe done in start() that shouldn't be.", queryId); } else { try { resultsWriter.close(); } catch (IOException e) { - log.warn(e, "Suppressing exception closing accumulator for query[%s]", queryId); + log.warn(e, "Suppressing exception closing accumulator for query [%s]", queryId); } } if (asyncContext != null) { @@ -197,58 +208,48 @@ public abstract class QueryResultPusher @Nullable private Response handleQueryException(ResultsWriter resultsWriter, QueryException e) { - if (accumulator != null && accumulator.isInitialized()) { - // We already started sending a response when we got the error message. In this case we just give up - // and hope that the partial stream generates a meaningful failure message for our client. We could consider - // also throwing the exception body into the response to make it easier for the client to choke if it manages - // to parse a meaningful object out, but that's potentially an API change so we leave that as an exercise for - // the future. + return handleDruidException(resultsWriter, DruidException.fromFailure(new QueryExceptionCompat(e))); + } + private Response handleDruidException(ResultsWriter resultsWriter, DruidException e) + { + if (resultsWriter != null) { resultsWriter.recordFailure(e); - - // This case is always a failure because the error happened mid-stream of sending results back. Therefore, - // we do not believe that the response stream was actually usable counter.incrementFailed(); - return null; + + if (accumulator != null && accumulator.isInitialized()) { + // We already started sending a response when we got the error message. In this case we just give up + // and hope that the partial stream generates a meaningful failure message for our client. We could consider + // also throwing the exception body into the response to make it easier for the client to choke if it manages + // to parse a meaningful object out, but that's potentially an API change so we leave that as an exercise for + // the future. + return null; + } } - final QueryException.FailType failType = e.getFailType(); - switch (failType) { - case USER_ERROR: + switch (e.getCategory()) { + case INVALID_INPUT: case UNAUTHORIZED: - case QUERY_RUNTIME_FAILURE: + case RUNTIME_FAILURE: case CANCELED: counter.incrementInterrupted(); break; case CAPACITY_EXCEEDED: case UNSUPPORTED: + case UNCATEGORIZED: + case DEFENSIVE: counter.incrementFailed(); break; case TIMEOUT: counter.incrementTimedOut(); break; - case UNKNOWN: - log.warn( - e, - "Unknown errorCode[%s], support needs to be added for error handling.", - e.getErrorCode() - ); - counter.incrementFailed(); } - resultsWriter.recordFailure(e); - - final int responseStatus = failType.getExpectedStatus(); - if (response == null) { - // No response object yet, so assume we haven't started the async context and is safe to return Response final Response.ResponseBuilder bob = Response - .status(responseStatus) + .status(e.getStatusCode()) .type(contentType) - .entity((StreamingOutput) output -> { - writeException(e, output); - output.close(); - }); + .entity(new ErrorResponse(e)); bob.header(QueryResource.QUERY_ID_RESPONSE_HEADER, queryId); for (Map.Entry entry : extraHeaders.entrySet()) { @@ -261,7 +262,7 @@ public abstract class QueryResultPusher QueryResource.NO_STACK_LOGGER.warn(e, "Response was committed without the accumulator writing anything!?"); } - response.setStatus(responseStatus); + response.setStatus(e.getStatusCode()); response.setHeader("Content-Type", contentType.toString()); try (ServletOutputStream out = response.getOutputStream()) { writeException(e, out); @@ -269,7 +270,7 @@ public abstract class QueryResultPusher catch (IOException ioException) { log.warn( ioException, - "Suppressing IOException thrown sending error response for query[%s]", + "Suppressing IOException thrown sending error response for query [%s]", queryId ); } diff --git a/server/src/main/java/org/apache/druid/server/security/AllowOptionsResourceFilter.java b/server/src/main/java/org/apache/druid/server/security/AllowOptionsResourceFilter.java index 46fe78a0a47..776f5b6df4d 100644 --- a/server/src/main/java/org/apache/druid/server/security/AllowOptionsResourceFilter.java +++ b/server/src/main/java/org/apache/druid/server/security/AllowOptionsResourceFilter.java @@ -44,7 +44,6 @@ public class AllowOptionsResourceFilter implements Filter @Override public void init(FilterConfig filterConfig) { - } @Override @@ -78,6 +77,5 @@ public class AllowOptionsResourceFilter implements Filter @Override public void destroy() { - } } diff --git a/server/src/main/java/org/apache/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/org/apache/druid/server/security/PreResponseAuthorizationCheckFilter.java index 97d3e05a5f3..454d8566f29 100644 --- a/server/src/main/java/org/apache/druid/server/security/PreResponseAuthorizationCheckFilter.java +++ b/server/src/main/java/org/apache/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -100,7 +100,7 @@ public class PreResponseAuthorizationCheckFilter implements Filter if (authInfoChecked != null && !authInfoChecked && response.getStatus() != HttpServletResponse.SC_FORBIDDEN) { handleAuthorizationCheckError( - "Request's authorization check failed but status code was not 403.", + "Request's authorization check failed but status code was not 403", request, response ); diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index ffc7934fafe..78294fca0c4 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -32,8 +32,10 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.DurationGranularity; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.jackson.JacksonUtils; @@ -48,6 +50,7 @@ import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -60,7 +63,6 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; -import java.util.List; import java.util.Map; import java.util.Set; @@ -413,22 +415,24 @@ public class DataSchemaTest extends InitializedNullHandlingTest ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - expectedException.expect(CoreMatchers.instanceOf(IllegalArgumentException.class)); - expectedException.expectMessage( - "dataSource cannot be null or empty. Please provide a dataSource." - ); - - DataSchema schema = new DataSchema( - "", - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DruidExceptionMatcher + .invalidInput() + .expectMessageIs("Invalid value for field [dataSource]: must not be null") + .assertThrowsAndMatches( + () -> new DataSchema( + "", + parser, + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2"), + }, + new ArbitraryGranularitySpec( + Granularities.DAY, + ImmutableList.of(Intervals.of("2014/2015")) + ), + null, + jsonMapper + )); } @@ -442,27 +446,21 @@ public class DataSchemaTest extends InitializedNullHandlingTest ); for (Map.Entry entry : invalidCharToDataSourceName.entrySet()) { - testInvalidWhitespaceDatasourceHelper(entry.getValue(), entry.getKey()); - } - } - - private void testInvalidWhitespaceDatasourceHelper(String dataSource, String invalidChar) - { - String testFailMsg = "dataSource contain invalid whitespace character: " + invalidChar; - try { - DataSchema schema = new DataSchema( - dataSource, - Collections.emptyMap(), - null, - null, - null, - jsonMapper + String dataSource = entry.getValue(); + final String msg = StringUtils.format( + "Invalid value for field [dataSource]: Value [%s] contains illegal whitespace characters. Only space is allowed.", + dataSource + ); + DruidExceptionMatcher.invalidInput().expectMessageIs(msg).assertThrowsAndMatches( + () -> new DataSchema( + dataSource, + Collections.emptyMap(), + null, + null, + null, + jsonMapper + ) ); - Assert.fail(testFailMsg); - } - catch (IllegalArgumentException errorMsg) { - String expectedMsg = "dataSource cannot contain whitespace character except space."; - Assert.assertEquals(testFailMsg, expectedMsg, errorMsg.getMessage()); } } @@ -524,10 +522,22 @@ public class DataSchemaTest extends InitializedNullHandlingTest public void testSerializeWithInvalidDataSourceName() throws Exception { // Escape backslashes to insert a tab character in the datasource name. - List datasources = ImmutableList.of("", "../invalid", "\tname", "name\t invalid"); - for (String datasource : datasources) { + Map datasourceToErrorMsg = ImmutableMap.of( + "", + "Invalid value for field [dataSource]: must not be null", + + "../invalid", + "Invalid value for field [dataSource]: Value [../invalid] cannot start with '.'.", + + "\tname", + "Invalid value for field [dataSource]: Value [\tname] contains illegal whitespace characters. Only space is allowed.", + + "name\t invalid", + "Invalid value for field [dataSource]: Value [name\t invalid] contains illegal whitespace characters. Only space is allowed." + ); + for (Map.Entry entry : datasourceToErrorMsg.entrySet()) { String jsonStr = "{" - + "\"dataSource\":\"" + StringEscapeUtils.escapeJson(datasource) + "\"," + + "\"dataSource\":\"" + StringEscapeUtils.escapeJson(entry.getKey()) + "\"," + "\"parser\":{" + "\"type\":\"string\"," + "\"parseSpec\":{" @@ -552,10 +562,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest ); } catch (ValueInstantiationException e) { - Assert.assertEquals(IllegalArgumentException.class, e.getCause().getClass()); + MatcherAssert.assertThat( + entry.getKey(), + e.getCause(), + DruidExceptionMatcher.invalidInput().expectMessageIs( + entry.getValue() + ) + ); continue; } - Assert.fail("Serialization of datasource " + datasource + " should have failed."); + Assert.fail("Serialization of datasource " + entry.getKey() + " should have failed."); } } diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 34c0a44a5cf..c7c96ecd3b1 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -30,6 +30,9 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Injector; import com.google.inject.Key; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.error.ErrorResponse; import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.jackson.DefaultObjectMapper; @@ -76,6 +79,7 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.server.security.Resource; import org.apache.http.HttpStatus; +import org.hamcrest.MatcherAssert; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -87,16 +91,18 @@ import javax.servlet.http.HttpServletResponse; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; -import javax.ws.rs.core.StreamingOutput; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -296,6 +302,79 @@ public class QueryResourceTest ); } + @Test + public void testGoodQueryThrowsDruidExceptionFromLifecycleExecute() throws IOException + { + String overrideConfigKey = "priority"; + String overrideConfigValue = "678"; + DefaultQueryConfig overrideConfig = new DefaultQueryConfig(ImmutableMap.of(overrideConfigKey, overrideConfigValue)); + queryResource = new QueryResource( + new QueryLifecycleFactory( + WAREHOUSE, + new QuerySegmentWalker() + { + @Override + public QueryRunner getQueryRunnerForIntervals( + Query query, + Iterable intervals + ) + { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("failing for coverage!"); + } + + @Override + public QueryRunner getQueryRunnerForSegments( + Query query, + Iterable specs + ) + { + throw new UnsupportedOperationException(); + } + }, + new DefaultGenericQueryMetricsFactory(), + new NoopServiceEmitter(), + testRequestLogger, + new AuthConfig(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + Suppliers.ofInstance(overrideConfig) + ), + jsonMapper, + smileMapper, + queryScheduler, + new AuthConfig(), + null, + ResponseContextConfig.newConfig(true), + DRUID_NODE + ); + + expectPermissiveHappyPathAuth(); + + final Response response = expectSynchronousRequestFlow(SIMPLE_TIMESERIES_QUERY); + Assert.assertEquals(Status.INTERNAL_SERVER_ERROR.getStatusCode(), response.getStatus()); + + final ErrorResponse entity = (ErrorResponse) response.getEntity(); + MatcherAssert.assertThat( + entity.getUnderlyingException(), + new DruidExceptionMatcher(DruidException.Persona.OPERATOR, DruidException.Category.RUNTIME_FAILURE, "general") + .expectMessageIs("failing for coverage!") + ); + + Assert.assertEquals(1, testRequestLogger.getNativeQuerylogs().size()); + Assert.assertNotNull(testRequestLogger.getNativeQuerylogs().get(0).getQuery()); + Assert.assertNotNull(testRequestLogger.getNativeQuerylogs().get(0).getQuery().getContext()); + Assert.assertTrue(testRequestLogger.getNativeQuerylogs() + .get(0) + .getQuery() + .getContext() + .containsKey(overrideConfigKey)); + Assert.assertEquals( + overrideConfigValue, + testRequestLogger.getNativeQuerylogs().get(0).getQuery().getContext().get(overrideConfigKey) + ); + } + @Test public void testGoodQueryWithQueryConfigDoesNotOverrideQueryContext() throws IOException { @@ -642,11 +721,23 @@ public class QueryResourceTest ); Assert.assertEquals(QueryTimeoutException.STATUS_CODE, response.getStatus()); + ErrorResponse entity = (ErrorResponse) response.getEntity(); + MatcherAssert.assertThat( + entity.getUnderlyingException(), + new DruidExceptionMatcher( + DruidException.Persona.OPERATOR, + DruidException.Category.TIMEOUT, + "legacyQueryException" + ) + .expectMessageIs( + "Query did not complete within configured timeout period. You can increase query timeout or tune the performance of query.") + ); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ((StreamingOutput) response.getEntity()).write(baos); + jsonMapper.writeValue(baos, entity); QueryTimeoutException ex = jsonMapper.readValue(baos.toByteArray(), QueryTimeoutException.class); Assert.assertEquals("Query did not complete within configured timeout period. You can " + - "increase query timeout or tune the performance of query.", ex.getMessage()); + "increase query timeout or tune the performance of query.", ex.getMessage()); Assert.assertEquals(QueryException.QUERY_TIMEOUT_ERROR_CODE, ex.getErrorCode()); Assert.assertEquals(1, timeoutQueryResource.getTimedOutQueryCount()); @@ -874,12 +965,11 @@ public class QueryResourceTest } @Test(timeout = 10_000L) - public void testTooManyQuery() throws InterruptedException + public void testTooManyQuery() throws InterruptedException, ExecutionException { expectPermissiveHappyPathAuth(); final CountDownLatch waitTwoScheduled = new CountDownLatch(2); - final CountDownLatch waitAllFinished = new CountDownLatch(3); final QueryScheduler laningScheduler = new QueryScheduler( 2, ManualQueryPrioritizationStrategy.INSTANCE, @@ -887,28 +977,42 @@ public class QueryResourceTest new ServerConfig() ); + ArrayList> back2 = new ArrayList<>(); + createScheduledQueryResource(laningScheduler, Collections.emptyList(), ImmutableList.of(waitTwoScheduled)); - assertAsyncResponseAndCountdownOrBlockForever( + back2.add(eventuallyAssertAsyncResponse( SIMPLE_TIMESERIES_QUERY, - waitAllFinished, response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) - ); - assertAsyncResponseAndCountdownOrBlockForever( + )); + back2.add(eventuallyAssertAsyncResponse( SIMPLE_TIMESERIES_QUERY, - waitAllFinished, response -> Assert.assertEquals(Status.OK.getStatusCode(), response.getStatus()) - ); + )); waitTwoScheduled.await(); - assertSynchronousResponseAndCountdownOrBlockForever( + back2.add(eventuallyaAssertSynchronousResponse( SIMPLE_TIMESERIES_QUERY, - waitAllFinished, response -> { Assert.assertEquals(QueryCapacityExceededException.STATUS_CODE, response.getStatus()); QueryCapacityExceededException ex; + final ErrorResponse entity = (ErrorResponse) response.getEntity(); + MatcherAssert.assertThat( + entity.getUnderlyingException(), + new DruidExceptionMatcher( + DruidException.Persona.OPERATOR, + DruidException.Category.CAPACITY_EXCEEDED, + "legacyQueryException" + ) + .expectMessageIs( + "Too many concurrent queries, total query capacity of 2 exceeded. Please try your query again later.") + ); + try { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ((StreamingOutput) response.getEntity()).write(baos); + jsonMapper.writeValue(baos, entity); + + // Here we are converting to a QueryCapacityExceededException. This is just to validate legacy stuff. + // When we delete the QueryException class, we can just rely on validating the DruidException instead ex = jsonMapper.readValue(baos.toByteArray(), QueryCapacityExceededException.class); } catch (IOException e) { @@ -917,17 +1021,19 @@ public class QueryResourceTest Assert.assertEquals(QueryCapacityExceededException.makeTotalErrorMessage(2), ex.getMessage()); Assert.assertEquals(QueryException.QUERY_CAPACITY_EXCEEDED_ERROR_CODE, ex.getErrorCode()); } - ); - waitAllFinished.await(); + )); + + for (Future theFuture : back2) { + Assert.assertTrue(theFuture.get()); + } } @Test(timeout = 10_000L) - public void testTooManyQueryInLane() throws InterruptedException + public void testTooManyQueryInLane() throws InterruptedException, ExecutionException { expectPermissiveHappyPathAuth(); final CountDownLatch waitTwoStarted = new CountDownLatch(2); final CountDownLatch waitOneScheduled = new CountDownLatch(1); - final CountDownLatch waitAllFinished = new CountDownLatch(3); final QueryScheduler scheduler = new QueryScheduler( 40, ManualQueryPrioritizationStrategy.INSTANCE, @@ -935,23 +1041,39 @@ public class QueryResourceTest new ServerConfig() ); + ArrayList> back2 = new ArrayList<>(); + createScheduledQueryResource(scheduler, ImmutableList.of(waitTwoStarted), ImmutableList.of(waitOneScheduled)); - assertAsyncResponseAndCountdownOrBlockForever( + back2.add(eventuallyAssertAsyncResponse( SIMPLE_TIMESERIES_QUERY_LOW_PRIORITY, - waitAllFinished, response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) - ); + )); waitOneScheduled.await(); - assertSynchronousResponseAndCountdownOrBlockForever( + back2.add(eventuallyaAssertSynchronousResponse( SIMPLE_TIMESERIES_QUERY_LOW_PRIORITY, - waitAllFinished, response -> { Assert.assertEquals(QueryCapacityExceededException.STATUS_CODE, response.getStatus()); QueryCapacityExceededException ex; + + final ErrorResponse entity = (ErrorResponse) response.getEntity(); + MatcherAssert.assertThat( + entity.getUnderlyingException(), + new DruidExceptionMatcher( + DruidException.Persona.OPERATOR, + DruidException.Category.CAPACITY_EXCEEDED, + "legacyQueryException" + ) + .expectMessageIs( + "Too many concurrent queries for lane 'low', query capacity of 1 exceeded. Please try your query again later.") + ); + try { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ((StreamingOutput) response.getEntity()).write(baos); + jsonMapper.writeValue(baos, entity); + + // Here we are converting to a QueryCapacityExceededException. This is just to validate legacy stuff. + // When we delete the QueryException class, we can just rely on validating the DruidException instead ex = jsonMapper.readValue(baos.toByteArray(), QueryCapacityExceededException.class); } catch (IOException e) { @@ -964,24 +1086,24 @@ public class QueryResourceTest Assert.assertEquals(QueryException.QUERY_CAPACITY_EXCEEDED_ERROR_CODE, ex.getErrorCode()); } - ); + )); waitTwoStarted.await(); - assertAsyncResponseAndCountdownOrBlockForever( + back2.add(eventuallyAssertAsyncResponse( SIMPLE_TIMESERIES_QUERY, - waitAllFinished, response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) - ); + )); - waitAllFinished.await(); + for (Future theFuture : back2) { + Assert.assertTrue(theFuture.get()); + } } @Test(timeout = 10_000L) - public void testTooManyQueryInLaneImplicitFromDurationThreshold() throws InterruptedException + public void testTooManyQueryInLaneImplicitFromDurationThreshold() throws InterruptedException, ExecutionException { expectPermissiveHappyPathAuth(); final CountDownLatch waitTwoStarted = new CountDownLatch(2); final CountDownLatch waitOneScheduled = new CountDownLatch(1); - final CountDownLatch waitAllFinished = new CountDownLatch(3); final QueryScheduler scheduler = new QueryScheduler( 40, new ThresholdBasedQueryPrioritizationStrategy(null, "P90D", null, null), @@ -989,23 +1111,38 @@ public class QueryResourceTest new ServerConfig() ); + ArrayList> back2 = new ArrayList<>(); createScheduledQueryResource(scheduler, ImmutableList.of(waitTwoStarted), ImmutableList.of(waitOneScheduled)); - assertAsyncResponseAndCountdownOrBlockForever( + back2.add(eventuallyAssertAsyncResponse( SIMPLE_TIMESERIES_QUERY, - waitAllFinished, response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) - ); + )); waitOneScheduled.await(); - assertSynchronousResponseAndCountdownOrBlockForever( + back2.add(eventuallyaAssertSynchronousResponse( SIMPLE_TIMESERIES_QUERY, - waitAllFinished, response -> { Assert.assertEquals(QueryCapacityExceededException.STATUS_CODE, response.getStatus()); QueryCapacityExceededException ex; + + final ErrorResponse entity = (ErrorResponse) response.getEntity(); + MatcherAssert.assertThat( + entity.getUnderlyingException(), + new DruidExceptionMatcher( + DruidException.Persona.OPERATOR, + DruidException.Category.CAPACITY_EXCEEDED, + "legacyQueryException" + ) + .expectMessageIs( + "Too many concurrent queries for lane 'low', query capacity of 1 exceeded. Please try your query again later.") + ); + try { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ((StreamingOutput) response.getEntity()).write(baos); + jsonMapper.writeValue(baos, entity); + + // Here we are converting to a QueryCapacityExceededException. This is just to validate legacy stuff. + // When we delete the QueryException class, we can just rely on validating the DruidException instead ex = jsonMapper.readValue(baos.toByteArray(), QueryCapacityExceededException.class); } catch (IOException e) { @@ -1017,15 +1154,16 @@ public class QueryResourceTest ); Assert.assertEquals(QueryException.QUERY_CAPACITY_EXCEEDED_ERROR_CODE, ex.getErrorCode()); } - ); + )); waitTwoStarted.await(); - assertAsyncResponseAndCountdownOrBlockForever( + back2.add(eventuallyAssertAsyncResponse( SIMPLE_TIMESERIES_QUERY_SMALLISH_INTERVAL, - waitAllFinished, response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) - ); + )); - waitAllFinished.await(); + for (Future theFuture : back2) { + Assert.assertTrue(theFuture.get()); + } } private void createScheduledQueryResource( @@ -1090,20 +1228,19 @@ public class QueryResourceTest ); } - private void assertAsyncResponseAndCountdownOrBlockForever( + private Future eventuallyAssertAsyncResponse( String query, - CountDownLatch done, Consumer asserts ) { - Executors.newSingleThreadExecutor().submit(() -> { + return Executors.newSingleThreadExecutor().submit(() -> { try { asserts.accept(expectAsyncRequestFlow(query, testServletRequest.mimic())); } catch (IOException e) { throw new RuntimeException(e); } - done.countDown(); + return true; }); } @@ -1139,7 +1276,8 @@ public class QueryResourceTest @Nonnull private MockHttpServletResponse expectAsyncRequestFlow( MockHttpServletRequest req, - byte[] queryBytes, QueryResource queryResource + byte[] queryBytes, + QueryResource queryResource ) throws IOException { final MockHttpServletResponse response = MockHttpServletResponse.forRequest(req); @@ -1152,13 +1290,12 @@ public class QueryResourceTest return response; } - private void assertSynchronousResponseAndCountdownOrBlockForever( + private Future eventuallyaAssertSynchronousResponse( String query, - CountDownLatch done, Consumer asserts ) { - Executors.newSingleThreadExecutor().submit(() -> { + return Executors.newSingleThreadExecutor().submit(() -> { try { asserts.accept( expectSynchronousRequestFlow( @@ -1171,10 +1308,19 @@ public class QueryResourceTest catch (IOException e) { throw new RuntimeException(e); } - done.countDown(); + return true; }); } + private Response expectSynchronousRequestFlow(String simpleTimeseriesQuery) throws IOException + { + return expectSynchronousRequestFlow( + testServletRequest, + simpleTimeseriesQuery.getBytes(StandardCharsets.UTF_8), + queryResource + ); + } + private Response expectSynchronousRequestFlow( MockHttpServletRequest req, byte[] bytes, diff --git a/server/src/test/java/org/apache/druid/server/security/AuthValidatorTest.java b/server/src/test/java/org/apache/druid/server/security/AuthValidatorTest.java index 3edfec1b5d0..c8f78fb15a5 100644 --- a/server/src/test/java/org/apache/druid/server/security/AuthValidatorTest.java +++ b/server/src/test/java/org/apache/druid/server/security/AuthValidatorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.server.security; +import org.apache.druid.error.DruidExceptionMatcher; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -40,41 +41,41 @@ public class AuthValidatorTest @Test public void testAuthorizerNameWithEmptyIsInvalid() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("authorizerName cannot be null or empty."); - target.validateAuthorizerName(""); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [authorizerName]: must not be null" + ).assertThrowsAndMatches(() -> target.validateAuthorizerName("")); } @Test public void testAuthorizerNameWithNullIsInvalid() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("authorizerName cannot be null or empty."); - target.validateAuthorizerName(null); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [authorizerName]: must not be null" + ).assertThrowsAndMatches(() -> target.validateAuthorizerName(null)); } @Test public void testAuthorizerNameStartsWithDotIsInValid() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("authorizerName cannot start with the '.' character."); - target.validateAuthorizerName(".test"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [authorizerName]: Value [.test] cannot start with '.'." + ).assertThrowsAndMatches(() -> target.validateAuthorizerName(".test")); } @Test public void testAuthorizerNameWithSlashIsInvalid() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("authorizerName cannot contain the '/' character."); - target.validateAuthorizerName("tes/t"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [authorizerName]: Value [tes/t] cannot contain '/'." + ).assertThrowsAndMatches(() -> target.validateAuthorizerName("tes/t")); } @Test public void testAuthorizerNameWithWhitespaceIsInvalid() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("authorizerName cannot contain whitespace character except space."); - target.validateAuthorizerName("tes\tt"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [authorizerName]: Value [tes\tt] contains illegal whitespace characters. Only space is allowed." + ).assertThrowsAndMatches(() -> target.validateAuthorizerName("tes\tt")); } @Test @@ -92,8 +93,8 @@ public class AuthValidatorTest @Test public void testAuthenticatorNameWithWhitespaceIsInvalid() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("authenticatorName cannot contain whitespace character except space."); - target.validateAuthenticatorName("tes\tt"); + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [authenticatorName]: Value [tes\tt] contains illegal whitespace characters. Only space is allowed." + ).assertThrowsAndMatches(() -> target.validateAuthenticatorName("tes\tt")); } } diff --git a/sql/pom.xml b/sql/pom.xml index eb2c1bf7a39..d032a64f5e2 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -229,6 +229,11 @@ test-jar test + + org.hamcrest + hamcrest-all + test + org.hamcrest hamcrest-core diff --git a/sql/src/main/codegen/includes/common.ftl b/sql/src/main/codegen/includes/common.ftl index 2eccdbc2a5a..8de677647be 100644 --- a/sql/src/main/codegen/includes/common.ftl +++ b/sql/src/main/codegen/includes/common.ftl @@ -65,7 +65,6 @@ org.apache.druid.java.util.common.Pair PartitionGranularity e = Expression(ExprContext.ACCEPT_SUB_QUERY) { granularity = DruidSqlParserUtils.convertSqlNodeToGranularityThrowingParseExceptions(e); - DruidSqlParserUtils.throwIfUnsupportedGranularityInPartitionedBy(granularity); unparseString = e.toString(); } ) diff --git a/sql/src/main/codegen/includes/insert.ftl b/sql/src/main/codegen/includes/insert.ftl index c0e04bc7724..a0482dbf8a6 100644 --- a/sql/src/main/codegen/includes/insert.ftl +++ b/sql/src/main/codegen/includes/insert.ftl @@ -38,7 +38,9 @@ SqlNode DruidSqlInsertEof() : ] { if (clusteredBy != null && partitionedBy.lhs == null) { - throw new ParseException("CLUSTERED BY found before PARTITIONED BY. In Druid, the CLUSTERED BY clause must follow the PARTITIONED BY clause"); + throw org.apache.druid.sql.calcite.parser.DruidSqlParserUtils.problemParsing( + "CLUSTERED BY found before PARTITIONED BY, CLUSTERED BY must come after the PARTITIONED BY clause" + ); } } // EOF is also present in SqlStmtEof but EOF is a special case and a single EOF can be consumed multiple times. diff --git a/sql/src/main/codegen/includes/replace.ftl b/sql/src/main/codegen/includes/replace.ftl index ed8dbb10eed..5d47c919581 100644 --- a/sql/src/main/codegen/includes/replace.ftl +++ b/sql/src/main/codegen/includes/replace.ftl @@ -58,7 +58,9 @@ SqlNode DruidSqlReplaceEof() : ] { if (clusteredBy != null && partitionedBy.lhs == null) { - throw new ParseException("CLUSTERED BY found before PARTITIONED BY. In Druid, the CLUSTERED BY clause must follow the PARTITIONED BY clause"); + throw org.apache.druid.sql.calcite.parser.DruidSqlParserUtils.problemParsing( + "CLUSTERED BY found before PARTITIONED BY, CLUSTERED BY must come after the PARTITIONED BY clause" + ); } } // EOF is also present in SqlStmtEof but EOF is a special case and a single EOF can be consumed multiple times. diff --git a/sql/src/main/java/org/apache/druid/sql/AbstractStatement.java b/sql/src/main/java/org/apache/druid/sql/AbstractStatement.java index 99d2fa17a61..bfa95c5d556 100644 --- a/sql/src/main/java/org/apache/druid/sql/AbstractStatement.java +++ b/sql/src/main/java/org/apache/druid/sql/AbstractStatement.java @@ -19,8 +19,6 @@ package org.apache.druid.sql; -import org.apache.calcite.sql.parser.SqlParseException; -import org.apache.calcite.tools.ValidationException; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QueryContexts; import org.apache.druid.server.security.Access; @@ -130,17 +128,7 @@ public abstract class AbstractStatement implements Closeable plannerContext = planner.getPlannerContext(); plannerContext.setAuthenticationResult(queryPlus.authResult()); plannerContext.setParameters(queryPlus.parameters()); - try { - planner.validate(); - } - // We can't collapse catch clauses since SqlPlanningException has - // type-sensitive constructors. - catch (SqlParseException e) { - throw new SqlPlanningException(e); - } - catch (ValidationException e) { - throw new SqlPlanningException(e); - } + planner.validate(); } /** diff --git a/sql/src/main/java/org/apache/druid/sql/DirectStatement.java b/sql/src/main/java/org/apache/druid/sql/DirectStatement.java index 62830063d21..0bcf0f684ca 100644 --- a/sql/src/main/java/org/apache/druid/sql/DirectStatement.java +++ b/sql/src/main/java/org/apache/druid/sql/DirectStatement.java @@ -20,7 +20,9 @@ package org.apache.druid.sql; import com.google.common.annotations.VisibleForTesting; -import org.apache.calcite.tools.ValidationException; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; @@ -225,11 +227,22 @@ public class DirectStatement extends AbstractStatement implements Cancelable reporter.planningTimeNanos(System.nanoTime() - planningStartNanos); return resultSet; } + catch (RelOptPlanner.CannotPlanException e) { + // Not sure if this is even thrown here. + throw DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Problem planning SQL query"); + } catch (RuntimeException e) { state = State.FAILED; reporter.failed(e); throw e; } + catch (AssertionError e) { + state = State.FAILED; + reporter.failed(e); + throw InvalidSqlInput.exception(e, "Calcite assertion violated: [%s]", e.getMessage()); + } } /** @@ -239,12 +252,7 @@ public class DirectStatement extends AbstractStatement implements Cancelable @VisibleForTesting protected PlannerResult createPlan(DruidPlanner planner) { - try { - return planner.plan(); - } - catch (ValidationException e) { - throw new SqlPlanningException(e); - } + return planner.plan(); } /** diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/AvgSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/AvgSqlAggregator.java index e7e252cf8e3..8d84ab30760 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/AvgSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/AvgSqlAggregator.java @@ -32,8 +32,8 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.Aggregations; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; @@ -95,12 +95,12 @@ public class AvgSqlAggregator implements SqlAggregator final DruidExpression arg = Iterables.getOnlyElement(arguments); final ExprMacroTable macroTable = plannerContext.getExprMacroTable(); - final ValueType sumType; + final ColumnType sumType; // Use 64-bit sum regardless of the type of the AVG aggregator. if (SqlTypeName.INT_TYPES.contains(aggregateCall.getType().getSqlTypeName())) { - sumType = ValueType.LONG; + sumType = ColumnType.LONG; } else { - sumType = ValueType.DOUBLE; + sumType = ColumnType.DOUBLE; } final String fieldName; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java index 6efc8846e91..0137689a851 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java @@ -35,8 +35,8 @@ import org.apache.calcite.sql.type.SqlReturnTypeInference; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeUtil; import org.apache.calcite.util.Optionality; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.any.DoubleAnyAggregatorFactory; import org.apache.druid.query.aggregation.any.FloatAnyAggregatorFactory; @@ -60,7 +60,6 @@ import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; import javax.annotation.Nullable; @@ -78,7 +77,13 @@ public class EarliestLatestAnySqlAggregator implements SqlAggregator { EARLIEST { @Override - AggregatorFactory createAggregatorFactory(String name, String fieldName, String timeColumn, ColumnType type, int maxStringBytes) + AggregatorFactory createAggregatorFactory( + String name, + String fieldName, + String timeColumn, + ColumnType type, + int maxStringBytes + ) { switch (type.getType()) { case LONG: @@ -91,14 +96,20 @@ public class EarliestLatestAnySqlAggregator implements SqlAggregator case COMPLEX: return new StringFirstAggregatorFactory(name, fieldName, timeColumn, maxStringBytes); default: - throw new UnsupportedSQLQueryException("EARLIEST aggregator is not supported for '%s' type", type); + throw SimpleSqlAggregator.badTypeException(fieldName, "EARLIEST", type); } } }, LATEST { @Override - AggregatorFactory createAggregatorFactory(String name, String fieldName, String timeColumn, ColumnType type, int maxStringBytes) + AggregatorFactory createAggregatorFactory( + String name, + String fieldName, + String timeColumn, + ColumnType type, + int maxStringBytes + ) { switch (type.getType()) { case LONG: @@ -111,14 +122,20 @@ public class EarliestLatestAnySqlAggregator implements SqlAggregator case COMPLEX: return new StringLastAggregatorFactory(name, fieldName, timeColumn, maxStringBytes); default: - throw new UnsupportedSQLQueryException("LATEST aggregator is not supported for '%s' type", type); + throw SimpleSqlAggregator.badTypeException(fieldName, "LATEST", type); } } }, ANY_VALUE { @Override - AggregatorFactory createAggregatorFactory(String name, String fieldName, String timeColumn, ColumnType type, int maxStringBytes) + AggregatorFactory createAggregatorFactory( + String name, + String fieldName, + String timeColumn, + ColumnType type, + int maxStringBytes + ) { switch (type.getType()) { case LONG: @@ -130,7 +147,7 @@ public class EarliestLatestAnySqlAggregator implements SqlAggregator case STRING: return new StringAnyAggregatorFactory(name, fieldName, maxStringBytes); default: - throw new UnsupportedSQLQueryException("ANY aggregation is not supported for '%s' type", type); + throw SimpleSqlAggregator.badTypeException(fieldName, "ANY", type); } } }; @@ -188,21 +205,30 @@ public class EarliestLatestAnySqlAggregator implements SqlAggregator final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name; final ColumnType outputType = Calcites.getColumnTypeForRelDataType(aggregateCall.getType()); if (outputType == null) { - throw new ISE( - "Cannot translate output sqlTypeName[%s] to Druid type for aggregator[%s]", - aggregateCall.getType().getSqlTypeName(), - aggregateCall.getName() - ); + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.DEFENSIVE) + .build( + "Cannot convert output SQL type[%s] to a Druid type for function [%s]", + aggregateCall.getName(), + aggregateCall.getType().getSqlTypeName() + ); } final String fieldName = getColumnName(plannerContext, virtualColumnRegistry, args.get(0), rexNodes.get(0)); if (!rowSignature.contains(ColumnHolder.TIME_COLUMN_NAME) && (aggregatorType == AggregatorType.LATEST || aggregatorType == AggregatorType.EARLIEST)) { - plannerContext.setPlanningError("%s() aggregator depends on __time column, the underlying datasource " - + "or extern function you are querying doesn't contain __time column, " - + "Please use %s_BY() and specify the time column you want to use", - aggregatorType.name(), - aggregatorType.name() + // This code is being run as part of the exploratory volcano planner, currently, the definition of these + // aggregators does not tell Calcite that they depend on a __time column being in existence, instead we are + // allowing the volcano planner to explore paths that put projections which eliminate the time column in between + // the table scan and the aggregation and then relying on this check to tell Calcite that the plan is bogus. + // In some future, it would be good to make the aggregator definition capable of telling Calcite that it depends + // on a __time column to be in existence. Or perhaps we should just kill these aggregators and have everything + // move to the _BY aggregators that require an explicit definition. Either way, for now, we set this potential + // error and let the volcano planner continue exploring + plannerContext.setPlanningError( + "LATEST and EARLIEST aggregators implicitly depend on the __time column, but the " + + "table queried doesn't contain a __time column. Please use LATEST_BY or EARLIEST_BY " + + "and specify the column explicitly." ); return null; } @@ -218,7 +244,11 @@ public class EarliestLatestAnySqlAggregator implements SqlAggregator maxStringBytes = RexLiteral.intValue(rexNodes.get(1)); } catch (AssertionError ae) { - plannerContext.setPlanningError("The second argument '%s' to function '%s' is not a number", rexNodes.get(1), aggregateCall.getName()); + plannerContext.setPlanningError( + "The second argument '%s' to function '%s' is not a number", + rexNodes.get(1), + aggregateCall.getName() + ); return null; } theAggFactory = aggregatorType.createAggregatorFactory( @@ -230,11 +260,10 @@ public class EarliestLatestAnySqlAggregator implements SqlAggregator ); break; default: - throw new IAE( - "aggregation[%s], Invalid number of arguments[%,d] to [%s] operator", - aggregatorName, - args.size(), - aggregatorType.name() + throw InvalidSqlInput.exception( + "Function [%s] expects 1 or 2 arguments but found [%s]", + aggregateCall.getName(), + args.size() ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MaxSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MaxSqlAggregator.java index e27b006778e..4f29b276a54 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MaxSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MaxSqlAggregator.java @@ -28,10 +28,8 @@ import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.FloatMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.planner.Calcites; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; public class MaxSqlAggregator extends SimpleSqlAggregator { @@ -53,17 +51,17 @@ public class MaxSqlAggregator extends SimpleSqlAggregator if (valueType == null) { return null; } - return Aggregation.create(createMaxAggregatorFactory(valueType.getType(), name, fieldName, macroTable)); + return Aggregation.create(createMaxAggregatorFactory(valueType, name, fieldName, macroTable)); } private static AggregatorFactory createMaxAggregatorFactory( - final ValueType aggregationType, + final ColumnType aggregationType, final String name, final String fieldName, final ExprMacroTable macroTable ) { - switch (aggregationType) { + switch (aggregationType.getType()) { case LONG: return new LongMaxAggregatorFactory(name, fieldName, null, macroTable); case FLOAT: @@ -71,7 +69,9 @@ public class MaxSqlAggregator extends SimpleSqlAggregator case DOUBLE: return new DoubleMaxAggregatorFactory(name, fieldName, null, macroTable); default: - throw new UnsupportedSQLQueryException("Max aggregation is not supported for '%s' type", aggregationType); + // This error refers to the Druid type. But, we're in SQL validation. + // It should refer to the SQL type. + throw SimpleSqlAggregator.badTypeException(fieldName, "MAX", aggregationType); } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MinSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MinSqlAggregator.java index b009ead1fe9..93b87d376b5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MinSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MinSqlAggregator.java @@ -30,7 +30,6 @@ import org.apache.druid.query.aggregation.LongMinAggregatorFactory; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.planner.Calcites; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; public class MinSqlAggregator extends SimpleSqlAggregator { @@ -67,7 +66,7 @@ public class MinSqlAggregator extends SimpleSqlAggregator case DOUBLE: return new DoubleMinAggregatorFactory(name, fieldName, null, macroTable); default: - throw new UnsupportedSQLQueryException("MIN aggregator is not supported for '%s' type", aggregationType); + throw SimpleSqlAggregator.badTypeException(fieldName, "MIN", aggregationType); } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SimpleSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SimpleSqlAggregator.java index 643a99394fe..3ac68483611 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SimpleSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SimpleSqlAggregator.java @@ -23,7 +23,10 @@ import com.google.common.collect.Iterables; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Project; import org.apache.calcite.rex.RexBuilder; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.Aggregations; @@ -45,6 +48,11 @@ import java.util.List; */ public abstract class SimpleSqlAggregator implements SqlAggregator { + public static DruidException badTypeException(String columnName, String agg, ColumnType type) + { + return InvalidSqlInput.exception("Aggregation [%s] does not support type [%s], column [%s]", agg, type, columnName); + } + @Nullable @Override public Aggregation toDruidAggregation( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java index 53d7fc4cf7a..4411ebd33cd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java @@ -27,6 +27,7 @@ import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlCallBinding; import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlOperatorBinding; @@ -51,7 +52,6 @@ import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; import org.apache.druid.sql.calcite.table.RowSignatures; @@ -197,7 +197,16 @@ public class StringSqlAggregator implements SqlAggregator { RelDataType type = sqlOperatorBinding.getOperandType(0); if (type instanceof RowSignatures.ComplexSqlType) { - throw new UnsupportedSQLQueryException("Cannot use STRING_AGG on complex inputs %s", type); + String columnName = ""; + if (sqlOperatorBinding instanceof SqlCallBinding) { + columnName = ((SqlCallBinding) sqlOperatorBinding).getCall().operand(0).toString(); + } + + throw SimpleSqlAggregator.badTypeException( + columnName, + "STRING_AGG", + ((RowSignatures.ComplexSqlType) type).getColumnType() + ); } return Calcites.createSqlTypeWithNullability( sqlOperatorBinding.getTypeFactory(), diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SumSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SumSqlAggregator.java index f4dcad3ed59..148c4dd0d0c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SumSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SumSqlAggregator.java @@ -37,10 +37,8 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.planner.Calcites; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; public class SumSqlAggregator extends SimpleSqlAggregator { @@ -70,17 +68,17 @@ public class SumSqlAggregator extends SimpleSqlAggregator if (valueType == null) { return null; } - return Aggregation.create(createSumAggregatorFactory(valueType.getType(), name, fieldName, macroTable)); + return Aggregation.create(createSumAggregatorFactory(valueType, name, fieldName, macroTable)); } static AggregatorFactory createSumAggregatorFactory( - final ValueType aggregationType, + final ColumnType aggregationType, final String name, final String fieldName, final ExprMacroTable macroTable ) { - switch (aggregationType) { + switch (aggregationType.getType()) { case LONG: return new LongSumAggregatorFactory(name, fieldName, null, macroTable); case FLOAT: @@ -88,7 +86,7 @@ public class SumSqlAggregator extends SimpleSqlAggregator case DOUBLE: return new DoubleSumAggregatorFactory(name, fieldName, null, macroTable); default: - throw new UnsupportedSQLQueryException("Sum aggregation is not supported for '%s' type", aggregationType); + throw SimpleSqlAggregator.badTypeException(fieldName, "SUM", aggregationType); } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java index 76640bcf809..c5503f7eb85 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java @@ -40,6 +40,8 @@ import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeTransforms; import org.apache.calcite.sql2rel.SqlRexConvertlet; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.Expr; @@ -56,10 +58,10 @@ import org.apache.druid.sql.calcite.expression.OperatorConversions; import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.apache.druid.sql.calcite.planner.convertlet.DruidConvertletFactory; import org.apache.druid.sql.calcite.table.RowSignatures; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.Collections; import java.util.List; @@ -195,17 +197,7 @@ public class NestedDataOperatorConversions } // pre-normalize path so that the same expressions with different jq syntax are collapsed final String path = (String) pathExpr.eval(InputBindings.nilBindings()).value(); - final List parts; - try { - parts = NestedPathFinder.parseJsonPath(path); - } - catch (IllegalArgumentException iae) { - throw new UnsupportedSQLQueryException( - "Cannot use [%s]: [%s]", - call.getOperator().getName(), - iae.getMessage() - ); - } + final List parts = extractNestedPathParts(call, path); final String jsonPath = NestedPathFinder.toNormalizedJsonPath(parts); final DruidExpression.ExpressionGenerator builder = (args) -> "json_query(" + args.get(0).getExpression() + ",'" + jsonPath + "')"; @@ -232,7 +224,6 @@ public class NestedDataOperatorConversions } } - /** * The {@link org.apache.calcite.sql2rel.StandardConvertletTable} converts json_value(.. RETURNING type) into * cast(json_value_any(..), type). @@ -386,17 +377,9 @@ public class NestedDataOperatorConversions } // pre-normalize path so that the same expressions with different jq syntax are collapsed final String path = (String) pathExpr.eval(InputBindings.nilBindings()).value(); - final List parts; - try { - parts = NestedPathFinder.parseJsonPath(path); - } - catch (IllegalArgumentException iae) { - throw new UnsupportedSQLQueryException( - "Cannot use [%s]: [%s]", - call.getOperator().getName(), - iae.getMessage() - ); - } + + final List parts = extractNestedPathParts(call, path); + final String jsonPath = NestedPathFinder.toNormalizedJsonPath(parts); final DruidExpression.ExpressionGenerator builder = (args) -> "json_value(" + args.get(0).getExpression() + ",'" + jsonPath + "', '" + druidType.asTypeString() + "')"; @@ -521,7 +504,7 @@ public class NestedDataOperatorConversions parts = NestedPathFinder.parseJsonPath(path); } catch (IllegalArgumentException iae) { - throw new UnsupportedSQLQueryException( + throw InvalidSqlInput.exception( "Cannot use [%s]: [%s]", call.getOperator().getName(), iae.getMessage() @@ -684,17 +667,7 @@ public class NestedDataOperatorConversions } // pre-normalize path so that the same expressions with different jq syntax are collapsed final String path = (String) pathExpr.eval(InputBindings.nilBindings()).value(); - final List parts; - try { - parts = NestedPathFinder.parseJsonPath(path); - } - catch (IllegalArgumentException iae) { - throw new UnsupportedSQLQueryException( - "Cannot use [%s]: [%s]", - call.getOperator().getName(), - iae.getMessage() - ); - } + final List parts = extractNestedPathParts(call, path); final String jsonPath = NestedPathFinder.toNormalizedJsonPath(parts); final DruidExpression.ExpressionGenerator builder = (args) -> "json_value(" + args.get(0).getExpression() + ",'" + jsonPath + "')"; @@ -897,4 +870,19 @@ public class NestedDataOperatorConversions ); } } + + @Nonnull + private static List extractNestedPathParts(RexCall call, String path) + { + try { + return NestedPathFinder.parseJsonPath(path); + } + catch (IllegalArgumentException iae) { + final String name = call.getOperator().getName(); + throw DruidException + .forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build(iae, "Error when processing path [%s], operator [%s] is not useable", path, name); + } + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java index 9009237b780..5f11c6f836a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java @@ -33,7 +33,8 @@ import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlTimestampLiteral; import org.apache.calcite.tools.ValidationException; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; @@ -65,7 +66,6 @@ import java.util.stream.Collectors; public class DruidSqlParserUtils { - private static final Logger log = new Logger(DruidSqlParserUtils.class); public static final String ALL = "all"; @@ -78,6 +78,9 @@ public class DruidSqlParserUtils try { return convertSqlNodeToGranularity(sqlNode); } + catch (DruidException e) { + throw e; + } catch (Exception e) { log.debug(e, StringUtils.format("Unable to convert %s to a valid granularity.", sqlNode.toString())); throw new ParseException(e.getMessage()); @@ -88,7 +91,7 @@ public class DruidSqlParserUtils * This method is used to extract the granularity from a SqlNode representing following function calls: * 1. FLOOR(__time TO TimeUnit) * 2. TIME_FLOOR(__time, 'PT1H') - * + *

* Validation on the sqlNode is contingent to following conditions: * 1. sqlNode is an instance of SqlCall * 2. Operator is either one of TIME_FLOOR or FLOOR @@ -96,7 +99,7 @@ public class DruidSqlParserUtils * 4. First operand is a SimpleIdentifier representing __time * 5. If operator is TIME_FLOOR, the second argument is a literal, and can be converted to the Granularity class * 6. If operator is FLOOR, the second argument is a TimeUnit, and can be mapped using {@link TimeUnits} - * + *

* Since it is to be used primarily while parsing the SqlNode, it is wrapped in {@code convertSqlNodeToGranularityThrowingParseExceptions} * * @param sqlNode SqlNode representing a call to a function @@ -105,16 +108,8 @@ public class DruidSqlParserUtils */ public static Granularity convertSqlNodeToGranularity(SqlNode sqlNode) throws ParseException { - - final String genericParseFailedMessageFormatString = "Encountered %s after PARTITIONED BY. " - + "Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR function or %s function"; - if (!(sqlNode instanceof SqlCall)) { - throw new ParseException(StringUtils.format( - genericParseFailedMessageFormatString, - sqlNode.toString(), - TimeFloorOperatorConversion.SQL_FUNCTION_NAME - )); + throw makeInvalidPartitionByException(sqlNode); } SqlCall sqlCall = (SqlCall) sqlNode; @@ -163,7 +158,9 @@ public class DruidSqlParserUtils catch (IllegalArgumentException e) { throw new ParseException(StringUtils.format("%s is an invalid period string", granularitySqlNode.toString())); } - return new PeriodGranularity(period, null, null); + final PeriodGranularity retVal = new PeriodGranularity(period, null, null); + validateSupportedGranularityForPartitionedBy(sqlNode, retVal); + return retVal; } else if ("FLOOR".equalsIgnoreCase(operatorName)) { // If the floor function is of form FLOOR(__time TO DAY) SqlNode granularitySqlNode = operandList.get(1); @@ -184,15 +181,22 @@ public class DruidSqlParserUtils granularityIntervalQualifier.timeUnitRange.toString() ) ); - return new PeriodGranularity(period, null, null); + final PeriodGranularity retVal = new PeriodGranularity(period, null, null); + validateSupportedGranularityForPartitionedBy(sqlNode, retVal); + return retVal; } // Shouldn't reach here - throw new ParseException(StringUtils.format( - genericParseFailedMessageFormatString, - sqlNode.toString(), - TimeFloorOperatorConversion.SQL_FUNCTION_NAME - )); + throw makeInvalidPartitionByException(sqlNode); + } + + private static DruidException makeInvalidPartitionByException(SqlNode sqlNode) + { + return InvalidSqlInput.exception( + "Invalid granularity [%s] after PARTITIONED BY. " + + "Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR() or TIME_FLOOR()", + sqlNode + ); } /** @@ -200,7 +204,7 @@ public class DruidSqlParserUtils * be used in creating an ingestion spec. If the sqlNode is an SqlLiteral of {@link #ALL}, returns a singleton list of * "ALL". Otherwise, it converts and optimizes the query using {@link MoveTimeFiltersToIntervals} into a list of * intervals which contain all valid values of time as per the query. - * + *

* The following validations are performed * 1. Only __time column and timestamp literals are present in the query * 2. The interval after optimization is not empty @@ -208,8 +212,8 @@ public class DruidSqlParserUtils * 4. The intervals after adjusting for timezone are aligned with the granularity parameter * * @param replaceTimeQuery Sql node representing the query - * @param granularity granularity of the query for validation - * @param dateTimeZone timezone + * @param granularity granularity of the query for validation + * @param dateTimeZone timezone * @return List of string representation of intervals * @throws ValidationException if the SqlNode cannot be converted to a list of intervals */ @@ -217,7 +221,7 @@ public class DruidSqlParserUtils SqlNode replaceTimeQuery, Granularity granularity, DateTimeZone dateTimeZone - ) throws ValidationException + ) { if (replaceTimeQuery instanceof SqlLiteral && ALL.equalsIgnoreCase(((SqlLiteral) replaceTimeQuery).toValue())) { return ImmutableList.of(ALL); @@ -230,19 +234,31 @@ public class DruidSqlParserUtils List intervals = filtration.getIntervals(); if (filtration.getDimFilter() != null) { - throw new ValidationException("Only " + ColumnHolder.TIME_COLUMN_NAME + " column is supported in OVERWRITE WHERE clause"); + throw InvalidSqlInput.exception( + "OVERWRITE WHERE clause only supports filtering on the __time column, got [%s]", + filtration.getDimFilter() + ); } if (intervals.isEmpty()) { - throw new ValidationException("Intervals for replace are empty"); + throw InvalidSqlInput.exception( + "The OVERWRITE WHERE clause [%s] produced no time intervals, are the bounds overly restrictive?", + dimFilter, + intervals + ); } for (Interval interval : intervals) { DateTime intervalStart = interval.getStart(); DateTime intervalEnd = interval.getEnd(); - if (!granularity.bucketStart(intervalStart).equals(intervalStart) || !granularity.bucketStart(intervalEnd).equals(intervalEnd)) { - throw new ValidationException("OVERWRITE WHERE clause contains an interval " + intervals + - " which is not aligned with PARTITIONED BY granularity " + granularity); + if (!granularity.bucketStart(intervalStart).equals(intervalStart) + || !granularity.bucketStart(intervalEnd).equals(intervalEnd)) { + throw InvalidSqlInput.exception( + "OVERWRITE WHERE clause identified interval [%s]" + + " which is not aligned with PARTITIONED BY granularity [%s]", + interval, + granularity + ); } } return intervals @@ -254,13 +270,12 @@ public class DruidSqlParserUtils /** * Extracts and converts the information in the CLUSTERED BY clause to a new SqlOrderBy node. * - * @param query sql query + * @param query sql query * @param clusteredByList List of clustered by columns * @return SqlOrderBy node containing the clusteredByList information * @throws ValidationException if any of the clustered by columns contain DESCENDING order. */ public static SqlOrderBy convertClusterByToOrderBy(SqlNode query, SqlNodeList clusteredByList) - throws ValidationException { validateClusteredByColumns(clusteredByList); // If we have a CLUSTERED BY clause, extract the information in that CLUSTERED BY and create a new @@ -290,10 +305,9 @@ public class DruidSqlParserUtils /** * Validates the clustered by columns to ensure that it does not contain DESCENDING order columns. * - * @param clusteredByNodes List of SqlNodes representing columns to be clustered by. - * @throws ValidationException if any of the clustered by columns contain DESCENDING order. + * @param clusteredByNodes List of SqlNodes representing columns to be clustered by. */ - public static void validateClusteredByColumns(final SqlNodeList clusteredByNodes) throws ValidationException + public static void validateClusteredByColumns(final SqlNodeList clusteredByNodes) { if (clusteredByNodes == null) { return; @@ -301,10 +315,9 @@ public class DruidSqlParserUtils for (final SqlNode clusteredByNode : clusteredByNodes.getList()) { if (clusteredByNode.isA(ImmutableSet.of(SqlKind.DESCENDING))) { - throw new ValidationException( - StringUtils.format("[%s] is invalid." - + " CLUSTERED BY columns cannot be sorted in descending order.", clusteredByNode.toString() - ) + throw InvalidSqlInput.exception( + "Invalid CLUSTERED BY clause [%s]: cannot sort in descending order.", + clusteredByNode ); } } @@ -316,111 +329,121 @@ public class DruidSqlParserUtils * are AND, OR, NOT, >, <, >=, <= and BETWEEN operators in the sql query. * * @param replaceTimeQuery Sql node representing the query - * @param dateTimeZone timezone + * @param dateTimeZone timezone * @return Dimfilter for the query * @throws ValidationException if the SqlNode cannot be converted a Dimfilter */ public static DimFilter convertQueryToDimFilter(SqlNode replaceTimeQuery, DateTimeZone dateTimeZone) - throws ValidationException { if (!(replaceTimeQuery instanceof SqlBasicCall)) { - log.error("Expected SqlBasicCall during parsing, but found " + replaceTimeQuery.getClass().getName()); - throw new ValidationException("Invalid OVERWRITE WHERE clause"); + throw InvalidSqlInput.exception( + "Invalid OVERWRITE WHERE clause [%s]: expected clause including AND, OR, NOT, >, <, >=, <= OR BETWEEN operators", + replaceTimeQuery + ); } - String columnName; - SqlBasicCall sqlBasicCall = (SqlBasicCall) replaceTimeQuery; - List operandList = sqlBasicCall.getOperandList(); - switch (sqlBasicCall.getOperator().getKind()) { - case AND: - List dimFilters = new ArrayList<>(); - for (SqlNode sqlNode : sqlBasicCall.getOperandList()) { - dimFilters.add(convertQueryToDimFilter(sqlNode, dateTimeZone)); - } - return new AndDimFilter(dimFilters); - case OR: - dimFilters = new ArrayList<>(); - for (SqlNode sqlNode : sqlBasicCall.getOperandList()) { - dimFilters.add(convertQueryToDimFilter(sqlNode, dateTimeZone)); - } - return new OrDimFilter(dimFilters); - case NOT: - return new NotDimFilter(convertQueryToDimFilter(sqlBasicCall.getOperandList().get(0), dateTimeZone)); - case GREATER_THAN_OR_EQUAL: - columnName = parseColumnName(operandList.get(0)); - return new BoundDimFilter( - columnName, - parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), - null, - false, - null, - null, - null, - StringComparators.NUMERIC - ); - case LESS_THAN_OR_EQUAL: - columnName = parseColumnName(operandList.get(0)); - return new BoundDimFilter( - columnName, - null, - parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), - null, - false, - null, - null, - StringComparators.NUMERIC - ); - case GREATER_THAN: - columnName = parseColumnName(operandList.get(0)); - return new BoundDimFilter( - columnName, - parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), - null, - true, - null, - null, - null, - StringComparators.NUMERIC - ); - case LESS_THAN: - columnName = parseColumnName(operandList.get(0)); - return new BoundDimFilter( - columnName, - null, - parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), - null, - true, - null, - null, - StringComparators.NUMERIC - ); - case BETWEEN: - columnName = parseColumnName(operandList.get(0)); - return new BoundDimFilter( - columnName, - parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), - parseTimeStampWithTimeZone(operandList.get(2), dateTimeZone), - false, - false, - null, - null, - StringComparators.NUMERIC - ); - default: - throw new ValidationException("Unsupported operation in OVERWRITE WHERE clause: " + sqlBasicCall.getOperator().getName()); + + try { + String columnName; + SqlBasicCall sqlBasicCall = (SqlBasicCall) replaceTimeQuery; + List operandList = sqlBasicCall.getOperandList(); + switch (sqlBasicCall.getOperator().getKind()) { + case AND: + List dimFilters = new ArrayList<>(); + for (SqlNode sqlNode : sqlBasicCall.getOperandList()) { + dimFilters.add(convertQueryToDimFilter(sqlNode, dateTimeZone)); + } + return new AndDimFilter(dimFilters); + case OR: + dimFilters = new ArrayList<>(); + for (SqlNode sqlNode : sqlBasicCall.getOperandList()) { + dimFilters.add(convertQueryToDimFilter(sqlNode, dateTimeZone)); + } + return new OrDimFilter(dimFilters); + case NOT: + return new NotDimFilter(convertQueryToDimFilter(sqlBasicCall.getOperandList().get(0), dateTimeZone)); + case GREATER_THAN_OR_EQUAL: + columnName = parseColumnName(operandList.get(0)); + return new BoundDimFilter( + columnName, + parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), + null, + false, + null, + null, + null, + StringComparators.NUMERIC + ); + case LESS_THAN_OR_EQUAL: + columnName = parseColumnName(operandList.get(0)); + return new BoundDimFilter( + columnName, + null, + parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), + null, + false, + null, + null, + StringComparators.NUMERIC + ); + case GREATER_THAN: + columnName = parseColumnName(operandList.get(0)); + return new BoundDimFilter( + columnName, + parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), + null, + true, + null, + null, + null, + StringComparators.NUMERIC + ); + case LESS_THAN: + columnName = parseColumnName(operandList.get(0)); + return new BoundDimFilter( + columnName, + null, + parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), + null, + true, + null, + null, + StringComparators.NUMERIC + ); + case BETWEEN: + columnName = parseColumnName(operandList.get(0)); + return new BoundDimFilter( + columnName, + parseTimeStampWithTimeZone(operandList.get(1), dateTimeZone), + parseTimeStampWithTimeZone(operandList.get(2), dateTimeZone), + false, + false, + null, + null, + StringComparators.NUMERIC + ); + default: + throw InvalidSqlInput.exception( + "Unsupported operation [%s] in OVERWRITE WHERE clause.", + sqlBasicCall.getOperator().getName() + ); + } + } + catch (DruidException e) { + throw e.prependAndBuild("Invalid OVERWRITE WHERE clause [%s]", replaceTimeQuery); } } /** * Converts a {@link SqlNode} identifier into a string representation * - * @param sqlNode the sql node + * @param sqlNode the SQL node * @return string representing the column name - * @throws ValidationException if the sql node is not an SqlIdentifier + * @throws DruidException if the SQL node is not an SqlIdentifier */ - public static String parseColumnName(SqlNode sqlNode) throws ValidationException + public static String parseColumnName(SqlNode sqlNode) { if (!(sqlNode instanceof SqlIdentifier)) { - throw new ValidationException("Expressions must be of the form __time TIMESTAMP"); + throw InvalidSqlInput.exception("Cannot parse column name from SQL expression [%s]", sqlNode); } return ((SqlIdentifier) sqlNode).getSimple(); } @@ -428,15 +451,15 @@ public class DruidSqlParserUtils /** * Converts a {@link SqlNode} into a timestamp, taking into account the timezone * - * @param sqlNode the sql node + * @param sqlNode the SQL node * @param timeZone timezone * @return the timestamp string as milliseconds from epoch - * @throws ValidationException if the sql node is not a SqlTimestampLiteral + * @throws DruidException if the SQL node is not a SqlTimestampLiteral */ - public static String parseTimeStampWithTimeZone(SqlNode sqlNode, DateTimeZone timeZone) throws ValidationException + private static String parseTimeStampWithTimeZone(SqlNode sqlNode, DateTimeZone timeZone) { if (!(sqlNode instanceof SqlTimestampLiteral)) { - throw new ValidationException("Expressions must be of the form __time TIMESTAMP"); + throw InvalidSqlInput.exception("Cannot get a timestamp from sql expression [%s]", sqlNode); } Timestamp sqlTimestamp = Timestamp.valueOf(((SqlTimestampLiteral) sqlNode).toFormattedString()); @@ -444,17 +467,12 @@ public class DruidSqlParserUtils return String.valueOf(zonedTimestamp.toInstant().toEpochMilli()); } - /** - * Throws an IAE with appropriate message if the granularity supplied is not present in - * {@link org.apache.druid.java.util.common.granularity.Granularities}. It also filters out NONE as it is not a valid - * granularity that can be supplied in PARTITIONED BY - */ - public static void throwIfUnsupportedGranularityInPartitionedBy(Granularity granularity) + public static void validateSupportedGranularityForPartitionedBy(SqlNode originalNode, Granularity granularity) { if (!GranularityType.isStandard(granularity)) { - throw new IAE( - "The granularity specified in PARTITIONED BY is not supported. " - + "Please use an equivalent of these granularities: %s.", + throw InvalidSqlInput.exception( + "The granularity specified in PARTITIONED BY [%s] is not supported. Valid options: [%s]", + originalNode == null ? granularity : originalNode, Arrays.stream(GranularityType.values()) .filter(granularityType -> !granularityType.equals(GranularityType.NONE)) .map(Enum::name) @@ -463,4 +481,9 @@ public class DruidSqlParserUtils ); } } + + public static DruidException problemParsing(String message) + { + return InvalidSqlInput.exception(message); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java index 59c4cca2851..fc6c08a0403 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java @@ -20,25 +20,33 @@ package org.apache.druid.sql.calcite.planner; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.runtime.CalciteContextException; import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.sql.SqlExplain; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.ValidationException; -import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.query.QueryContext; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.parser.DruidSqlReplace; +import org.apache.druid.sql.calcite.parser.ParseException; +import org.apache.druid.sql.calcite.parser.Token; import org.apache.druid.sql.calcite.run.SqlEngine; import org.joda.time.DateTimeZone; import java.io.Closeable; +import java.util.ArrayList; import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -56,6 +64,10 @@ import java.util.function.Function; */ public class DruidPlanner implements Closeable { + + public static final Joiner SPACE_JOINER = Joiner.on(" "); + public static final Joiner COMMA_JOINER = Joiner.on(", "); + public enum State { START, VALIDATED, PREPARED, PLANNED @@ -119,7 +131,7 @@ public class DruidPlanner implements Closeable * @return set of {@link Resource} corresponding to any Druid datasources * or views which are taking part in the query. */ - public void validate() throws SqlParseException, ValidationException + public void validate() { Preconditions.checkState(state == State.START); @@ -129,22 +141,21 @@ public class DruidPlanner implements Closeable // Parse the query string. String sql = plannerContext.getSql(); hook.captureSql(sql); - SqlNode root = planner.parse(sql); - handler = createHandler(root); - + SqlNode root; try { - handler.validate(); - plannerContext.setResourceActions(handler.resourceActions()); - plannerContext.setExplainAttributes(handler.explainAttributes()); + root = planner.parse(sql); } - catch (RuntimeException e) { - throw new ValidationException(e); + catch (SqlParseException e1) { + throw translateException(e1); } - + handler = createHandler(root); + handler.validate(); + plannerContext.setResourceActions(handler.resourceActions()); + plannerContext.setExplainAttributes(handler.explainAttributes()); state = State.VALIDATED; } - private SqlStatementHandler createHandler(final SqlNode node) throws ValidationException + private SqlStatementHandler createHandler(final SqlNode node) { SqlNode query = node; SqlExplain explain = null; @@ -165,17 +176,17 @@ public class DruidPlanner implements Closeable if (query.isA(SqlKind.QUERY)) { return new QueryHandler.SelectHandler(handlerContext, query, explain); } - throw new ValidationException(StringUtils.format("Cannot execute [%s].", node.getKind())); + throw InvalidSqlInput.exception("Unsupported SQL statement [%s]", node.getKind()); } /** * Prepare a SQL query for execution, including some initial parsing and * validation and any dynamic parameter type resolution, to support prepared * statements via JDBC. - * + *

* Prepare reuses the validation done in {@link #validate()} which must be * called first. - * + *

* A query can be prepared on a data source without having permissions on * that data source. This odd state of affairs is necessary because * {@link org.apache.druid.sql.calcite.view.DruidViewMacro} prepares @@ -193,11 +204,10 @@ public class DruidPlanner implements Closeable * Authorizes the statement. Done within the planner to enforce the authorization * step within the planner's state machine. * - * @param authorizer a function from resource actions to a {@link Access} result. + * @param authorizer a function from resource actions to a {@link Access} result. * @param extraActions set of additional resource actions beyond those inferred - * from the query itself. Specifically, the set of context keys to - * authorize. - * + * from the query itself. Specifically, the set of context keys to + * authorize. * @return the return value from the authorizer */ public AuthResult authorize( @@ -227,7 +237,7 @@ public class DruidPlanner implements Closeable *

* Planning reuses the validation done in {@code validate()} which must be called first. */ - public PlannerResult plan() throws ValidationException + public PlannerResult plan() { Preconditions.checkState(state == State.VALIDATED || state == State.PREPARED); Preconditions.checkState(authorized); @@ -307,4 +317,186 @@ public class DruidPlanner implements Closeable return hook; } } + + public static DruidException translateException(Exception e) + { + try { + throw e; + } + catch (DruidException inner) { + return inner; + } + catch (ValidationException inner) { + return parseValidationMessage(inner); + } + catch (SqlParseException inner) { + final Throwable cause = inner.getCause(); + if (cause instanceof DruidException) { + return (DruidException) cause; + } + + if (cause instanceof ParseException) { + ParseException parseException = (ParseException) cause; + final SqlParserPos failurePosition = inner.getPos(); + final String theUnexpectedToken = getUnexpectedTokenString(parseException); + + final String[] tokenDictionary = inner.getTokenImages(); + final int[][] expectedTokenSequences = inner.getExpectedTokenSequences(); + final ArrayList expectedTokens = new ArrayList<>(expectedTokenSequences.length); + for (int[] expectedTokenSequence : expectedTokenSequences) { + String[] strings = new String[expectedTokenSequence.length]; + for (int i = 0; i < expectedTokenSequence.length; ++i) { + strings[i] = tokenDictionary[expectedTokenSequence[i]]; + } + expectedTokens.add(SPACE_JOINER.join(strings)); + } + + return InvalidSqlInput + .exception( + inner, + "Received an unexpected token [%s] (line [%s], column [%s]), acceptable options: [%s]", + theUnexpectedToken, + failurePosition.getLineNum(), + failurePosition.getColumnNum(), + COMMA_JOINER.join(expectedTokens) + ) + .withContext("line", failurePosition.getLineNum()) + .withContext("column", failurePosition.getColumnNum()) + .withContext("endLine", failurePosition.getEndLineNum()) + .withContext("endColumn", failurePosition.getEndColumnNum()) + .withContext("token", theUnexpectedToken) + .withContext("expected", expectedTokens); + + } + + return DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build( + inner, + "Unable to parse the SQL, unrecognized error from calcite: [%s]", + inner.getMessage() + ); + } + catch (RelOptPlanner.CannotPlanException inner) { + return DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build(inner, inner.getMessage()); + } + catch (Exception inner) { + // Anything else. Should not get here. Anything else should already have + // been translated to a DruidException unless it is an unexpected exception. + return DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(inner, inner.getMessage()); + } + } + + private static DruidException parseValidationMessage(Exception e) + { + if (e.getCause() instanceof DruidException) { + return (DruidException) e.getCause(); + } + + Throwable maybeContextException = e; + CalciteContextException contextException = null; + while (maybeContextException != null) { + if (maybeContextException instanceof CalciteContextException) { + contextException = (CalciteContextException) maybeContextException; + break; + } + maybeContextException = maybeContextException.getCause(); + } + + if (contextException != null) { + return InvalidSqlInput + .exception( + e, + "%s (line [%s], column [%s])", + // the CalciteContextException .getMessage() assumes cause is non-null, so this should be fine + contextException.getCause().getMessage(), + contextException.getPosLine(), + contextException.getPosColumn() + ) + .withContext("line", String.valueOf(contextException.getPosLine())) + .withContext("column", String.valueOf(contextException.getPosColumn())) + .withContext("endLine", String.valueOf(contextException.getEndPosLine())) + .withContext("endColumn", String.valueOf(contextException.getEndPosColumn())); + } else { + return DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Uncategorized calcite error message: [%s]", e.getMessage()); + } + } + + /** + * Grabs the unexpected token string. This code is borrowed with minimal adjustments from + * {@link ParseException#getMessage()}. It is possible that if that code changes, we need to also + * change this code to match it. + * + * @param parseException the parse exception to extract from + * @return the String representation of the unexpected token string + */ + private static String getUnexpectedTokenString(ParseException parseException) + { + int maxSize = 0; + for (int[] ints : parseException.expectedTokenSequences) { + if (maxSize < ints.length) { + maxSize = ints.length; + } + } + + StringBuilder bob = new StringBuilder(); + Token tok = parseException.currentToken.next; + for (int i = 0; i < maxSize; i++) { + if (i != 0) { + bob.append(" "); + } + if (tok.kind == 0) { + bob.append(""); + break; + } + char ch; + for (int i1 = 0; i1 < tok.image.length(); i1++) { + switch (tok.image.charAt(i1)) { + case 0: + continue; + case '\b': + bob.append("\\b"); + continue; + case '\t': + bob.append("\\t"); + continue; + case '\n': + bob.append("\\n"); + continue; + case '\f': + bob.append("\\f"); + continue; + case '\r': + bob.append("\\r"); + continue; + case '\"': + bob.append("\\\""); + continue; + case '\'': + bob.append("\\\'"); + continue; + case '\\': + bob.append("\\\\"); + continue; + default: + if ((ch = tok.image.charAt(i1)) < 0x20 || ch > 0x7e) { + String s = "0000" + Integer.toString(ch, 16); + bob.append("\\u").append(s.substring(s.length() - 4, s.length())); + } else { + bob.append(ch); + } + continue; + } + } + tok = tok.next; + } + return bob.toString(); + } + } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java index 6b9e42bafd0..844d9896ae8 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java @@ -23,6 +23,7 @@ import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexExecutor; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; @@ -87,7 +88,7 @@ public class DruidRexExecutor implements RexExecutor // as a primitive long/float/double. // ExprEval.isNumericNull checks whether the parsed primitive value is null or not. if (!constExp.getType().isNullable() && exprResult.isNumericNull()) { - throw new UnsupportedSQLQueryException("Illegal DATE constant: %s", constExp); + throw InvalidSqlInput.exception("Illegal DATE constant [%s]", constExp); } literal = rexBuilder.makeDateLiteral( @@ -101,7 +102,7 @@ public class DruidRexExecutor implements RexExecutor // as a primitive long/float/double. // ExprEval.isNumericNull checks whether the parsed primitive value is null or not. if (!constExp.getType().isNullable() && exprResult.isNumericNull()) { - throw new UnsupportedSQLQueryException("Illegal TIMESTAMP constant: %s", constExp); + throw InvalidSqlInput.exception("Illegal TIMESTAMP constant [%s]", constExp); } literal = Calcites.jodaToCalciteTimestampLiteral( @@ -125,12 +126,12 @@ public class DruidRexExecutor implements RexExecutor // the query can execute. double exprResultDouble = exprResult.asDouble(); if (Double.isNaN(exprResultDouble) || Double.isInfinite(exprResultDouble)) { - String expression = druidExpression.getExpression(); - throw new UnsupportedSQLQueryException("'%s' evaluates to '%s' that is not supported in SQL. You can either cast the expression as BIGINT ('CAST(%s as BIGINT)') or VARCHAR ('CAST(%s as VARCHAR)') or change the expression itself", - expression, - Double.toString(exprResultDouble), - expression, - expression); + throw InvalidSqlInput.exception( + "Expression [%s] evaluates to an unsupported value [%s], expected something that" + + " can be a Double. Consider casting with 'CAST( AS BIGINT)'", + druidExpression.getExpression(), + exprResultDouble + ); } bigDecimal = BigDecimal.valueOf(exprResult.asDouble()); } @@ -160,12 +161,14 @@ public class DruidRexExecutor implements RexExecutor if (doubleVal == null) { resultAsBigDecimalList.add(null); } else if (Double.isNaN(doubleVal.doubleValue()) || Double.isInfinite(doubleVal.doubleValue())) { - String expression = druidExpression.getExpression(); - throw new UnsupportedSQLQueryException( - "'%s' contains an element that evaluates to '%s' which is not supported in SQL. You can either cast the element in the ARRAY to BIGINT or VARCHAR or change the expression itself", - expression, - Double.toString(doubleVal.doubleValue()) - ); + throw InvalidSqlInput.exception( + "Expression [%s] was expected to generate values that are all Doubles," + + " but entry at index[%d] was not: [%s]." + + " Consider Casting values to ensure a consistent type.", + druidExpression.getExpression(), + resultAsBigDecimalList.size(), + doubleVal + ); } else { resultAsBigDecimalList.add(BigDecimal.valueOf(doubleVal.doubleValue())); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java index 459784eb3bb..52b4efcfeb0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java @@ -20,7 +20,6 @@ package org.apache.druid.sql.calcite.planner; import com.fasterxml.jackson.core.JsonProcessingException; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.rel.RelRoot; @@ -36,7 +35,8 @@ import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.tools.ValidationException; import org.apache.calcite.util.Pair; import org.apache.druid.common.utils.IdUtils; -import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; @@ -55,12 +55,6 @@ import java.util.regex.Pattern; public abstract class IngestHandler extends QueryHandler { private static final Pattern UNNAMED_COLUMN_PATTERN = Pattern.compile("^EXPR\\$\\d+$", Pattern.CASE_INSENSITIVE); - @VisibleForTesting - public static final String UNNAMED_INGESTION_COLUMN_ERROR = - "Cannot ingest expressions that do not have an alias " - + "or columns with names like EXPR$[digit].\n" - + "E.g. if you are ingesting \"func(X)\", then you can rewrite it as " - + "\"func(X) as myColumn\""; protected final Granularity ingestionGranularity; protected String targetDatasource; @@ -77,7 +71,7 @@ public abstract class IngestHandler extends QueryHandler handlerContext.hook().captureInsert(ingestNode); } - protected static SqlNode convertQuery(DruidSqlIngest sqlNode) throws ValidationException + protected static SqlNode convertQuery(DruidSqlIngest sqlNode) { SqlNode query = sqlNode.getSource(); @@ -86,12 +80,10 @@ public abstract class IngestHandler extends QueryHandler SqlOrderBy sqlOrderBy = (SqlOrderBy) query; SqlNodeList orderByList = sqlOrderBy.orderList; if (!(orderByList == null || orderByList.equals(SqlNodeList.EMPTY))) { - String opName = sqlNode.getOperator().getName(); - throw new ValidationException(StringUtils.format( - "Cannot have ORDER BY on %s %s statement, use CLUSTERED BY instead.", - "INSERT".equals(opName) ? "an" : "a", - opName - )); + throw InvalidSqlInput.exception( + "Cannot use an ORDER BY clause on a Query of type [%s], use CLUSTERED BY instead", + sqlNode.getOperator().getName() + ); } } if (sqlNode.getClusteredBy() != null) { @@ -99,7 +91,7 @@ public abstract class IngestHandler extends QueryHandler } if (!query.isA(SqlKind.QUERY)) { - throw new ValidationException(StringUtils.format("Cannot execute [%s].", query.getKind())); + throw InvalidSqlInput.exception("Unexpected SQL statement type [%s], expected it to be a QUERY", query.getKind()); } return query; } @@ -112,13 +104,13 @@ public abstract class IngestHandler extends QueryHandler protected abstract DruidSqlIngest ingestNode(); @Override - public void validate() throws ValidationException + public void validate() { if (ingestNode().getPartitionedBy() == null) { - throw new ValidationException(StringUtils.format( - "%s statements must specify PARTITIONED BY clause explicitly", + throw InvalidSqlInput.exception( + "Operation [%s] requires a PARTITIONED BY to be explicitly defined, but none was found.", operationName() - )); + ); } try { PlannerContext plannerContext = handlerContext.plannerContext(); @@ -130,18 +122,16 @@ public abstract class IngestHandler extends QueryHandler } } catch (JsonProcessingException e) { - throw new ValidationException("Unable to serialize partition granularity."); + throw InvalidSqlInput.exception(e, "Invalid partition granularity [%s]", ingestionGranularity); } super.validate(); // Check if CTX_SQL_OUTER_LIMIT is specified and fail the query if it is. CTX_SQL_OUTER_LIMIT being provided causes // the number of rows inserted to be limited which is likely to be confusing and unintended. if (handlerContext.queryContextMap().get(PlannerContext.CTX_SQL_OUTER_LIMIT) != null) { - throw new ValidationException( - StringUtils.format( - "%s cannot be provided with %s.", - PlannerContext.CTX_SQL_OUTER_LIMIT, - operationName() - ) + throw InvalidSqlInput.exception( + "Context parameter [%s] cannot be provided on operator [%s]", + PlannerContext.CTX_SQL_OUTER_LIMIT, + operationName() ); } targetDatasource = validateAndGetDataSourceForIngest(); @@ -154,22 +144,27 @@ public abstract class IngestHandler extends QueryHandler final RelDataTypeFactory typeFactory = rootQueryRel.rel.getCluster().getTypeFactory(); return handlerContext.engine().resultTypeForInsert( typeFactory, - rootQueryRel.validatedRowType); + rootQueryRel.validatedRowType + ); } /** * Extract target datasource from a {@link SqlInsert}, and also validate that the ingestion is of a form we support. * Expects the target datasource to be either an unqualified name, or a name qualified by the default schema. */ - private String validateAndGetDataSourceForIngest() throws ValidationException + private String validateAndGetDataSourceForIngest() { final SqlInsert insert = ingestNode(); if (insert.isUpsert()) { - throw new ValidationException("UPSERT is not supported."); + throw InvalidSqlInput.exception("UPSERT is not supported."); } if (insert.getTargetColumnList() != null) { - throw new ValidationException(operationName() + " with a target column list is not supported."); + throw InvalidSqlInput.exception( + "Operation [%s] cannot be run with a target column list, given [%s (%s)]", + operationName(), + insert.getTargetTable(), insert.getTargetColumnList() + ); } final SqlIdentifier tableIdentifier = (SqlIdentifier) insert.getTargetTable(); @@ -177,7 +172,9 @@ public abstract class IngestHandler extends QueryHandler if (tableIdentifier.names.isEmpty()) { // I don't think this can happen, but include a branch for it just in case. - throw new ValidationException(operationName() + " requires a target table."); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.DEFENSIVE) + .build("Operation [%s] requires a target table", operationName()); } else if (tableIdentifier.names.size() == 1) { // Unqualified name. dataSource = Iterables.getOnlyElement(tableIdentifier.names); @@ -189,22 +186,15 @@ public abstract class IngestHandler extends QueryHandler if (tableIdentifier.names.size() == 2 && defaultSchemaName.equals(tableIdentifier.names.get(0))) { dataSource = tableIdentifier.names.get(1); } else { - throw new ValidationException( - StringUtils.format( - "Cannot %s into %s because it is not a Druid datasource.", - operationName(), - tableIdentifier - ) + throw InvalidSqlInput.exception( + "Table [%s] does not support operation [%s] because it is not a Druid datasource", + tableIdentifier, + operationName() ); } } - try { - IdUtils.validateId(operationName() + " dataSource", dataSource); - } - catch (IllegalArgumentException e) { - throw new ValidationException(e.getMessage()); - } + IdUtils.validateId("table", dataSource); return dataSource; } @@ -222,15 +212,20 @@ public abstract class IngestHandler extends QueryHandler return handlerContext.engine().buildQueryMakerForInsert( targetDatasource, rootQueryRel, - handlerContext.plannerContext()); + handlerContext.plannerContext() + ); } - private void validateColumnsForIngestion(RelRoot rootQueryRel) throws ValidationException + private void validateColumnsForIngestion(RelRoot rootQueryRel) { // Check that there are no unnamed columns in the insert. for (Pair field : rootQueryRel.fields) { if (UNNAMED_COLUMN_PATTERN.matcher(field.right).matches()) { - throw new ValidationException(UNNAMED_INGESTION_COLUMN_ERROR); + throw InvalidSqlInput.exception( + "Insertion requires columns to be named, but at least one of the columns was unnamed. This is usually " + + "the result of applying a function without having an AS clause, please ensure that all function calls" + + "are named with an AS clause as in \"func(X) as myColumn\"." + ); } } } @@ -246,13 +241,14 @@ public abstract class IngestHandler extends QueryHandler SqlStatementHandler.HandlerContext handlerContext, DruidSqlInsert sqlNode, SqlExplain explain - ) throws ValidationException + ) { super( handlerContext, sqlNode, convertQuery(sqlNode), - explain); + explain + ); this.sqlNode = sqlNode; } @@ -263,12 +259,12 @@ public abstract class IngestHandler extends QueryHandler } @Override - public void validate() throws ValidationException + public void validate() { if (!handlerContext.plannerContext().featureAvailable(EngineFeature.CAN_INSERT)) { - throw new ValidationException(StringUtils.format( - "Cannot execute INSERT with SQL engine '%s'.", - handlerContext.engine().name()) + throw InvalidSqlInput.exception( + "INSERT operations are not supported by requested SQL engine [%s], consider using MSQ.", + handlerContext.engine().name() ); } super.validate(); @@ -299,7 +295,7 @@ public abstract class IngestHandler extends QueryHandler SqlStatementHandler.HandlerContext handlerContext, DruidSqlReplace sqlNode, SqlExplain explain - ) throws ValidationException + ) { super( handlerContext, @@ -317,24 +313,27 @@ public abstract class IngestHandler extends QueryHandler } @Override - public void validate() throws ValidationException + public void validate() { if (!handlerContext.plannerContext().featureAvailable(EngineFeature.CAN_REPLACE)) { - throw new ValidationException(StringUtils.format( - "Cannot execute REPLACE with SQL engine '%s'.", - handlerContext.engine().name()) + throw InvalidSqlInput.exception( + "REPLACE operations are not supported by the requested SQL engine [%s]. Consider using MSQ.", + handlerContext.engine().name() ); } SqlNode replaceTimeQuery = sqlNode.getReplaceTimeQuery(); if (replaceTimeQuery == null) { - throw new ValidationException("Missing time chunk information in OVERWRITE clause for REPLACE. Use " - + "OVERWRITE WHERE <__time based condition> or OVERWRITE ALL to overwrite the entire table."); + throw InvalidSqlInput.exception( + "Missing time chunk information in OVERWRITE clause for REPLACE. Use " + + "OVERWRITE WHERE <__time based condition> or OVERWRITE ALL to overwrite the entire table." + ); } replaceIntervals = DruidSqlParserUtils.validateQueryAndConvertToIntervals( replaceTimeQuery, ingestionGranularity, - handlerContext.timeZone()); + handlerContext.timeZone() + ); super.validate(); if (replaceIntervals != null) { handlerContext.queryContextMap().put( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java index 4395da5fe44..691c33567a8 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java @@ -30,13 +30,11 @@ import org.apache.calcite.config.CalciteConnectionConfigImpl; import org.apache.calcite.plan.Context; import org.apache.calcite.plan.ConventionTraitDef; import org.apache.calcite.rel.RelCollationTraitDef; -import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.validate.SqlConformance; import org.apache.calcite.sql2rel.SqlToRelConverter; import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.Frameworks; -import org.apache.calcite.tools.ValidationException; import org.apache.druid.guice.annotations.Json; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.segment.join.JoinableFactoryWrapper; @@ -126,12 +124,7 @@ public class PlannerFactory extends PlannerToolbox final DruidPlanner thePlanner = createPlanner(engine, sql, queryContext, null); thePlanner.getPlannerContext() .setAuthenticationResult(NoopEscalator.getInstance().createEscalatedAuthenticationResult()); - try { - thePlanner.validate(); - } - catch (SqlParseException | ValidationException e) { - throw new RuntimeException(e); - } + thePlanner.validate(); thePlanner.authorize(ra -> Access.OK, ImmutableSet.of()); return thePlanner; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index 45b4390d0ba..c11d600e262 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -54,10 +54,10 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.validate.SqlValidator; import org.apache.calcite.tools.ValidationException; import org.apache.calcite.util.Pair; -import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.Query; import org.apache.druid.server.QueryResponse; @@ -106,10 +106,15 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand } @Override - public void validate() throws ValidationException + public void validate() { CalcitePlanner planner = handlerContext.planner(); - validatedQueryNode = planner.validate(rewriteParameters()); + try { + validatedQueryNode = planner.validate(rewriteParameters()); + } + catch (ValidationException e) { + throw DruidPlanner.translateException(e); + } final SqlValidator validator = planner.getValidator(); SqlResourceCollectorShuttle resourceCollectorShuttle = new SqlResourceCollectorShuttle( @@ -183,7 +188,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand } @Override - public PlannerResult plan() throws ValidationException + public PlannerResult plan() { prepare(); final Set bindableTables = getBindableTables(rootQueryRel.rel); @@ -196,15 +201,12 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand // Consider BINDABLE convention when necessary. Used for metadata tables. if (!handlerContext.plannerContext().featureAvailable(EngineFeature.ALLOW_BINDABLE_PLAN)) { - throw new ValidationException( - StringUtils.format( - "Cannot query table%s %s with SQL engine '%s'.", - bindableTables.size() != 1 ? "s" : "", - bindableTables.stream() - .map(table -> Joiner.on(".").join(table.getQualifiedName())) - .collect(Collectors.joining(", ")), - handlerContext.engine().name() - ) + throw InvalidSqlInput.exception( + "Cannot query table(s) [%s] with SQL engine [%s]", + bindableTables.stream() + .map(table -> Joiner.on(".").join(table.getQualifiedName())) + .collect(Collectors.joining(", ")), + handlerContext.engine().name() ); } @@ -214,20 +216,30 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand return planForDruid(); } } - catch (Exception e) { - Throwable cannotPlanException = Throwables.getCauseOfType(e, RelOptPlanner.CannotPlanException.class); - if (null == cannotPlanException) { - // Not a CannotPlanException, rethrow without logging. + catch (RelOptPlanner.CannotPlanException e) { + throw buildSQLPlanningError(e); + } + catch (RuntimeException e) { + if (e instanceof DruidException) { throw e; } - Logger logger = log; - if (!handlerContext.queryContext().isDebug()) { - logger = log.noStackTrace(); + // Calcite throws a Runtime exception as the result of an IllegalTargetException + // as the result of invoking a method dynamically, when that method throws an + // exception. Unwrap the exception if this exception is from Calcite. + RelOptPlanner.CannotPlanException cpe = Throwables.getCauseOfType(e, RelOptPlanner.CannotPlanException.class); + if (cpe != null) { + throw buildSQLPlanningError(cpe); } - String errorMessage = buildSQLPlanningErrorMessage(cannotPlanException); - logger.warn(e, errorMessage); - throw new UnsupportedSQLQueryException(errorMessage); + DruidException de = Throwables.getCauseOfType(e, DruidException.class); + if (de != null) { + throw de; + } + throw DruidPlanner.translateException(e); + } + catch (Exception e) { + // Not sure what this is. Should it have been translated sooner? + throw DruidPlanner.translateException(e); } } @@ -274,10 +286,10 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand * things that are not directly translatable to native Druid queries such * as system tables and just a general purpose (but definitely not optimized) * fall-back. - * + *

* See {@link #planWithDruidConvention} which will handle things which are * directly translatable to native Druid queries. - * + *

* The bindable path handles parameter substitution of any values not * bound by the earlier steps. */ @@ -313,43 +325,43 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand } else { final BindableRel theRel = bindableRel; final DataContext dataContext = plannerContext.createDataContext( - planner.getTypeFactory(), - plannerContext.getParameters() + planner.getTypeFactory(), + plannerContext.getParameters() ); final Supplier> resultsSupplier = () -> { final Enumerable enumerable = theRel.bind(dataContext); final Enumerator enumerator = enumerable.enumerator(); return QueryResponse.withEmptyContext( Sequences.withBaggage(new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public QueryHandler.EnumeratorIterator make() + new BaseSequence.IteratorMaker>() { - return new QueryHandler.EnumeratorIterator<>(new Iterator() + @Override + public QueryHandler.EnumeratorIterator make() { - @Override - public boolean hasNext() + return new QueryHandler.EnumeratorIterator<>(new Iterator() { - return enumerator.moveNext(); - } + @Override + public boolean hasNext() + { + return enumerator.moveNext(); + } - @Override - public Object[] next() - { - return (Object[]) enumerator.current(); - } - }); + @Override + public Object[] next() + { + return (Object[]) enumerator.current(); + } + }); + } + + @Override + public void cleanup(QueryHandler.EnumeratorIterator iterFromMake) + { + + } } - - @Override - public void cleanup(QueryHandler.EnumeratorIterator iterFromMake) - { - - } - } - ), enumerator::close) - ); + ), enumerator::close) + ); }; return new PlannerResult(resultsSupplier, rootQueryRel.validatedRowType); } @@ -562,12 +574,11 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand * This method wraps the root with a {@link LogicalSort} that applies a limit (no ordering change). If the outer rel * is already a {@link Sort}, we can merge our outerLimit into it, similar to what is going on in * {@link org.apache.druid.sql.calcite.rule.SortCollapseRule}. - * + *

* The {@link PlannerContext#CTX_SQL_OUTER_LIMIT} flag that controls this wrapping is meant for internal use only by * the web console, allowing it to apply a limit to queries without rewriting the original SQL. * * @param root root node - * * @return root node wrapped with a limiting logical sort if a limit is specified in the query context. */ @Nullable @@ -611,23 +622,28 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand protected abstract QueryMaker buildQueryMaker(RelRoot rootQueryRel) throws ValidationException; - private String buildSQLPlanningErrorMessage(Throwable exception) + private DruidException buildSQLPlanningError(RelOptPlanner.CannotPlanException exception) { String errorMessage = handlerContext.plannerContext().getPlanningError(); if (null == errorMessage && exception instanceof UnsupportedSQLQueryException) { errorMessage = exception.getMessage(); } - if (null == errorMessage) { - errorMessage = "Please check Broker logs for additional details."; + if (errorMessage == null) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build(exception, "Unhandled Query Planning Failure, see broker logs for details"); } else { // Planning errors are more like hints: it isn't guaranteed that the planning error is actually what went wrong. - errorMessage = "Possible error: " + errorMessage; + // For this reason, we consider these as targetting a more expert persona, i.e. the admin instead of the actual + // user. + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + exception, + "Query planning failed for unknown reason, our best guess is this [%s]", + errorMessage + ); } - // Finally, add the query itself to error message that user will get. - return StringUtils.format( - "Query not supported. %s SQL was: %s", errorMessage, - handlerContext.plannerContext().getSql() - ); } public static class SelectHandler extends QueryHandler @@ -635,19 +651,17 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand public SelectHandler( HandlerContext handlerContext, SqlNode sqlNode, - SqlExplain explain) + SqlExplain explain + ) { super(handlerContext, sqlNode, explain); } @Override - public void validate() throws ValidationException + public void validate() { if (!handlerContext.plannerContext().featureAvailable(EngineFeature.CAN_SELECT)) { - throw new ValidationException(StringUtils.format( - "Cannot execute SELECT with SQL engine '%s'.", - handlerContext.engine().name()) - ); + throw InvalidSqlInput.exception("Cannot execute SELECT with SQL engine [%s]", handlerContext.engine().name()); } super.validate(); } @@ -673,7 +687,8 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand { return handlerContext.engine().buildQueryMakerForSelect( rootQueryRel, - handlerContext.plannerContext()); + handlerContext.plannerContext() + ); } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/RelParameterizerShuttle.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/RelParameterizerShuttle.java index cd9b1c2d213..4fc27d3af40 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/RelParameterizerShuttle.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/RelParameterizerShuttle.java @@ -43,9 +43,8 @@ import org.apache.calcite.rex.RexDynamicParam; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexShuttle; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.sql.SqlPlanningException; -import org.apache.druid.sql.SqlPlanningException.PlanningError; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; /** * Traverse {@link RelNode} tree and replaces all {@link RexDynamicParam} with {@link org.apache.calcite.rex.RexLiteral} @@ -201,10 +200,7 @@ public class RelParameterizerShuttle implements RelShuttle if (plannerContext.getParameters().size() > dynamicParam.getIndex()) { TypedValue param = plannerContext.getParameters().get(dynamicParam.getIndex()); if (param == null) { - throw new SqlPlanningException( - PlanningError.VALIDATION_ERROR, - StringUtils.format("Parameter at position [%s] is not bound", dynamicParam.getIndex()) - ); + throw unbound(dynamicParam); } if (param.value == null) { return builder.makeNullLiteral(typeFactory.createSqlType(SqlTypeName.NULL)); @@ -216,12 +212,14 @@ public class RelParameterizerShuttle implements RelShuttle true ); } else { - throw new SqlPlanningException( - PlanningError.VALIDATION_ERROR, - StringUtils.format("Parameter at position [%s] is not bound", dynamicParam.getIndex()) - ); + throw unbound(dynamicParam); } } return node; } + + private static DruidException unbound(RexDynamicParam dynamicParam) + { + return InvalidSqlInput.exception("No value bound for parameter (position [%s])", dynamicParam.getIndex() + 1); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java index a0c9fcd7285..6619f48704e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java @@ -28,7 +28,8 @@ import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.util.SqlShuttle; import org.apache.calcite.util.TimestampString; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidSqlInput; import java.util.ArrayList; import java.util.Arrays; @@ -38,7 +39,7 @@ import java.util.List; * Replaces all {@link SqlDynamicParam} encountered in an {@link SqlNode} tree * with a {@link SqlLiteral} if a value binding exists for the parameter, if * possible. This is used in tandem with {@link RelParameterizerShuttle}. - * + *

* It is preferable that all parameters are placed here to pick up as many * optimizations as possible, but the facilities to convert jdbc types to * {@link SqlLiteral} are a bit less rich here than exist for converting a @@ -46,7 +47,7 @@ import java.util.List; * {@link org.apache.calcite.rex.RexLiteral}, which is why * {@link SqlParameterizerShuttle} and {@link RelParameterizerShuttle} * both exist. - * + *

* As it turns out, most parameters will be replaced in this shuttle. * The one exception are DATE types expressed as integers. For reasons * known only to Calcite, the {@code RexBuilder.clean()} method, used by @@ -69,11 +70,11 @@ public class SqlParameterizerShuttle extends SqlShuttle public SqlNode visit(SqlDynamicParam param) { if (plannerContext.getParameters().size() <= param.getIndex()) { - throw new IAE("Parameter at position [%s] is not bound", param.getIndex()); + throw unbound(param); } TypedValue paramBinding = plannerContext.getParameters().get(param.getIndex()); if (paramBinding == null) { - throw new IAE("Parameter at position [%s] is not bound", param.getIndex()); + throw unbound(param); } if (paramBinding.value == null) { return SqlLiteral.createNull(param.getParserPosition()); @@ -91,7 +92,7 @@ public class SqlParameterizerShuttle extends SqlShuttle } if (typeName == SqlTypeName.ARRAY) { - return createArrayLiteral(paramBinding.value); + return createArrayLiteral(paramBinding.value, param.getIndex()); } try { // This throws ClassCastException for a DATE parameter given as @@ -105,6 +106,11 @@ public class SqlParameterizerShuttle extends SqlShuttle } } + private static DruidException unbound(SqlDynamicParam param) + { + return InvalidSqlInput.exception("No value bound for parameter (position [%s])", param.getIndex() + 1); + } + /** * Convert an ARRAY parameter to the equivalent of the ARRAY[a, b, ...] * syntax. This is not well-supported in the present version of Calcite, @@ -112,7 +118,7 @@ public class SqlParameterizerShuttle extends SqlShuttle * structure. Supports a limited set of member types. Does not attempt * to enforce that all elements have the same type. */ - private SqlNode createArrayLiteral(Object value) + private SqlNode createArrayLiteral(Object value, int posn) { List list; if (value instanceof List) { @@ -121,9 +127,10 @@ public class SqlParameterizerShuttle extends SqlShuttle list = Arrays.asList((Object[]) value); } List args = new ArrayList<>(list.size()); - for (Object element : list) { + for (int i = 0, listSize = list.size(); i < listSize; i++) { + Object element = list.get(i); if (element == null) { - throw new IAE("An array parameter cannot contain null values"); + throw InvalidSqlInput.exception("parameter [%d] is an array, with an illegal null at index [%d]", posn + 1, i); } SqlNode node; if (element instanceof String) { @@ -135,9 +142,11 @@ public class SqlParameterizerShuttle extends SqlShuttle } else if (element instanceof Boolean) { node = SqlLiteral.createBoolean((Boolean) value, SqlParserPos.ZERO); } else { - throw new IAE( - "An array parameter does not allow values of type %s", - value.getClass().getSimpleName() + throw InvalidSqlInput.exception( + "parameter [%d] is an array, with an illegal value of type [%s] at index [%d]", + posn + 1, + value.getClass(), + i ); } args.add(node); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlStatementHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlStatementHandler.java index 4cb263c5220..267feae1f0f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlStatementHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlStatementHandler.java @@ -21,7 +21,6 @@ package org.apache.druid.sql.calcite.planner; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.calcite.schema.SchemaPlus; -import org.apache.calcite.tools.ValidationException; import org.apache.druid.query.QueryContext; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.calcite.run.SqlEngine; @@ -36,11 +35,11 @@ import java.util.Set; */ public interface SqlStatementHandler { - void validate() throws ValidationException; + void validate(); Set resourceActions(); void prepare(); PrepareResult prepareResult(); - PlannerResult plan() throws ValidationException; + PlannerResult plan(); ExplainAttributes explainAttributes(); /** diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java index 5ab29ab13b1..4ea14c3a741 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java @@ -38,6 +38,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -53,7 +54,6 @@ import org.apache.druid.sql.calcite.expression.Expressions; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerContext; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.apache.druid.sql.calcite.table.RowSignatures; import javax.annotation.Nullable; @@ -360,7 +360,10 @@ public class DruidJoinQueryRel extends DruidRel case INNER: return JoinType.INNER; default: - throw new UnsupportedSQLQueryException("Cannot handle joinType '%s'", calciteJoinType); + throw InvalidSqlInput.exception( + "Cannot handle joinType [%s]", + calciteJoinType + ); } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 1f71a9212cb..d0fcacf5df2 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -1430,12 +1430,14 @@ public class DruidQuery } if (!plannerContext.featureAvailable(EngineFeature.SCAN_ORDER_BY_NON_TIME) && !orderByColumns.isEmpty()) { - if (orderByColumns.size() > 1 || orderByColumns.stream() - .anyMatch(orderBy -> !orderBy.getColumnName().equals(ColumnHolder.TIME_COLUMN_NAME))) { - // Cannot handle this ordering. - // Scan cannot ORDER BY non-time columns. + if (orderByColumns.size() > 1 || !ColumnHolder.TIME_COLUMN_NAME.equals(orderByColumns.get(0).getColumnName())) { + // We cannot handle this ordering, but we encounter this ordering as part of the exploration of the volcano + // planner, which means that the query that we are looking right now might only be doing this as one of the + // potential branches of exploration rather than being a semantic requirement of the query itself. So, it is + // not safe to send an error message telling the end-user exactly what is happening, instead we need to set the + // planning error and hope. plannerContext.setPlanningError( - "SQL query requires order by non-time column %s, which is not supported.", + "SQL query requires order by non-time column [%s], which is not supported.", orderByColumns ); return null; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRule.java index b94a6ee4ac8..ea71dfd9098 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRule.java @@ -25,11 +25,11 @@ import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.rel.logical.LogicalValues; import org.apache.calcite.rex.RexLiteral; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.query.InlineDataSource; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.apache.druid.sql.calcite.rel.DruidQueryRel; import org.apache.druid.sql.calcite.table.RowSignatures; @@ -42,7 +42,7 @@ import java.util.stream.Collectors; * This rule is used when the query directly reads in-memory tuples. For example, given a query of * `SELECT 1 + 1`, the query planner will create {@link LogicalValues} that contains one tuple, * which in turn containing one column of value 2. - * + *

* The query planner can sometimes reduce a regular query to a query that reads in-memory tuples. * For example, `SELECT count(*) FROM foo WHERE 1 = 0` is reduced to `SELECT 0`. This rule will * be used for this case as well. @@ -126,14 +126,18 @@ public class DruidLogicalValuesRule extends RelOptRule return Calcites.calciteDateTimeLiteralToJoda(literal, plannerContext.getTimeZone()).getMillis(); case NULL: if (!literal.isNull()) { - throw new UnsupportedSQLQueryException("Query has a non-null constant but is of NULL type."); + throw InvalidSqlInput.exception("Expected a NULL literal, but got non-null constant [%s]", literal); } return null; case TIMESTAMP_WITH_LOCAL_TIME_ZONE: case TIME: case TIME_WITH_LOCAL_TIME_ZONE: default: - throw new UnsupportedSQLQueryException("%s type is not supported", literal.getType().getSqlTypeName()); + throw InvalidSqlInput.exception( + "Cannot handle literal [%s] of unsupported type [%s].", + literal, + literal.getType().getSqlTypeName() + ); } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java index 3d952accda3..f5d9056246b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java @@ -25,10 +25,9 @@ import com.google.inject.Inject; import org.apache.calcite.rel.RelRoot; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.tools.ValidationException; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.guice.LazySingleton; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.timeboundary.TimeBoundaryQuery; import org.apache.druid.server.QueryLifecycleFactory; @@ -77,7 +76,7 @@ public class NativeSqlEngine implements SqlEngine } @Override - public void validateContext(Map queryContext) throws ValidationException + public void validateContext(Map queryContext) { SqlEngines.validateNoSpecialContextKeys(queryContext, SYSTEM_CONTEXT_PARAMETERS); validateJoinAlgorithm(queryContext); @@ -146,18 +145,12 @@ public class NativeSqlEngine implements SqlEngine * Validates that {@link PlannerContext#CTX_SQL_JOIN_ALGORITHM} is {@link JoinAlgorithm#BROADCAST}. This is the * only join algorithm supported by native queries. */ - private static void validateJoinAlgorithm(final Map queryContext) throws ValidationException + private static void validateJoinAlgorithm(final Map queryContext) { final JoinAlgorithm joinAlgorithm = PlannerContext.getJoinAlgorithm(queryContext); if (joinAlgorithm != JoinAlgorithm.BROADCAST) { - throw new ValidationException( - StringUtils.format( - "Join algorithm [%s] is not supported by engine [%s]", - joinAlgorithm, - NAME - ) - ); + throw InvalidSqlInput.exception("Join algorithm [%s] is not supported by engine [%s]", joinAlgorithm, NAME); } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java index 22c8545dd67..678ded23e9d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java @@ -46,7 +46,7 @@ public interface SqlEngine * Validates a provided query context. Returns quietly if the context is OK; throws {@link ValidationException} * if the context has a problem. */ - void validateContext(Map queryContext) throws ValidationException; + void validateContext(Map queryContext); /** * SQL row type that would be emitted by the {@link QueryMaker} from {@link #buildQueryMakerForSelect}. diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngines.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngines.java index 30dd7926bd2..cc7bef80f71 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngines.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngines.java @@ -20,7 +20,7 @@ package org.apache.druid.sql.calcite.run; import org.apache.calcite.tools.ValidationException; -import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.error.InvalidInput; import java.util.Map; import java.util.Set; @@ -35,17 +35,14 @@ public class SqlEngines * * This is a helper function used by {@link SqlEngine#validateContext} implementations. */ - public static void validateNoSpecialContextKeys(final Map queryContext, final Set specialContextKeys) - throws ValidationException + public static void validateNoSpecialContextKeys( + final Map queryContext, + final Set specialContextKeys + ) { for (String contextParameterName : queryContext.keySet()) { if (specialContextKeys.contains(contextParameterName)) { - throw new ValidationException( - StringUtils.format( - "Cannot execute query with context parameter [%s]", - contextParameterName - ) - ); + throw InvalidInput.exception("Query context parameter [%s] is not allowed", contextParameterName); } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java b/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java index 13e2c268d06..32abe56ee8d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java @@ -220,6 +220,11 @@ public class RowSignatures sb.append(columnType.asTypeString()); } + public ColumnType getColumnType() + { + return columnType; + } + public String getComplexTypeName() { return columnType.getComplexTypeName(); diff --git a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java index dad391e6bf0..4adea5d8d84 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java @@ -22,13 +22,11 @@ package org.apache.druid.sql.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.inject.Inject; -import org.apache.calcite.plan.RelOptPlanner; import org.apache.druid.common.exception.SanitizableException; import org.apache.druid.guice.annotations.NativeQuery; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryResource; import org.apache.druid.server.QueryResponse; @@ -43,7 +41,6 @@ import org.apache.druid.sql.DirectStatement.ResultSet; import org.apache.druid.sql.HttpStatement; import org.apache.druid.sql.SqlLifecycleManager; import org.apache.druid.sql.SqlLifecycleManager.Cancelable; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.SqlRowTransformer; import org.apache.druid.sql.SqlStatementFactory; @@ -102,7 +99,6 @@ public class SqlResource this.serverConfig = Preconditions.checkNotNull(serverConfig, "serverConfig"); this.responseContextConfig = responseContextConfig; this.selfNode = selfNode; - } @POST @@ -177,25 +173,21 @@ public class SqlResource @Override public void incrementSuccess() { - } @Override public void incrementFailed() { - } @Override public void incrementInterrupted() { - } @Override public void incrementTimedOut() { - } } @@ -214,7 +206,6 @@ public class SqlResource private class SqlResourceQueryResultPusher extends QueryResultPusher { - private final String sqlQueryId; private final HttpStatement stmt; private final SqlQuery sqlQuery; @@ -229,9 +220,9 @@ public class SqlResource { super( req, - SqlResource.this.jsonMapper, - SqlResource.this.responseContextConfig, - SqlResource.this.selfNode, + jsonMapper, + responseContextConfig, + selfNode, SqlResource.QUERY_METRIC_COUNTER, sqlQueryId, MediaType.APPLICATION_JSON_TYPE, @@ -254,28 +245,9 @@ public class SqlResource @Nullable public Response.ResponseBuilder start() { - try { - thePlan = stmt.plan(); - queryResponse = thePlan.run(); - return null; - } - catch (RelOptPlanner.CannotPlanException e) { - throw new SqlPlanningException( - SqlPlanningException.PlanningError.UNSUPPORTED_SQL_ERROR, - e.getMessage() - ); - } - // There is a claim that Calcite sometimes throws a java.lang.AssertionError, but we do not have a test that can - // reproduce it checked into the code (the best we have is something that uses mocks to throw an Error, which is - // dubious at best). We keep this just in case, but it might be best to remove it and see where the - // AssertionErrors are coming from and do something to ensure that they don't actually make it out of Calcite - catch (AssertionError e) { - log.warn(e, "AssertionError killed query: %s", sqlQuery); - - // We wrap the exception here so that we get the sanitization. java.lang.AssertionError apparently - // doesn't implement org.apache.druid.common.exception.SanitizableException. - throw new QueryInterruptedException(e); - } + thePlan = stmt.plan(); + queryResponse = thePlan.run(); + return null; } @Override @@ -368,6 +340,5 @@ public class SqlResource } out.write(jsonMapper.writeValueAsBytes(ex)); } - } } diff --git a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java index 4d96a2ec908..c9a100e6a05 100644 --- a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java @@ -25,6 +25,8 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.LazySequence; @@ -47,7 +49,6 @@ import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.DirectStatement.ResultSet; -import org.apache.druid.sql.SqlPlanningException.PlanningError; import org.apache.druid.sql.calcite.planner.CalciteRulesManager; import org.apache.druid.sql.calcite.planner.CatalogResolver; import org.apache.druid.sql.calcite.planner.DruidOperatorTable; @@ -60,11 +61,13 @@ import org.apache.druid.sql.calcite.util.QueryLogHook; import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.http.SqlQuery; import org.easymock.EasyMock; +import org.hamcrest.MatcherAssert; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -85,12 +88,12 @@ import static org.junit.Assert.fail; public class SqlStatementTest { private static QueryRunnerFactoryConglomerate conglomerate; + private static SpecificSegmentsQuerySegmentWalker walker; private static Closer resourceCloser; - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @Rule public QueryLogHook queryLogHook = QueryLogHook.create(); - private SpecificSegmentsQuerySegmentWalker walker = null; private TestRequestLogger testRequestLogger; private ListeningExecutorService executorService; private SqlStatementFactory sqlStatementFactory; @@ -98,21 +101,11 @@ public class SqlStatementTest ImmutableMap.of("DEFAULT_KEY", "DEFAULT_VALUE")); @BeforeClass - public static void setUpClass() + public static void setUpClass() throws Exception { resourceCloser = Closer.create(); conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser); - } - @AfterClass - public static void tearDownClass() throws IOException - { - resourceCloser.close(); - } - - @Before - public void setUp() throws Exception - { final QueryScheduler scheduler = new QueryScheduler( 5, ManualQueryPrioritizationStrategy.INSTANCE, @@ -125,15 +118,25 @@ public class SqlStatementTest { return super.run( query, - new LazySequence(() -> { - return resultSequence; - }) + new LazySequence(() -> resultSequence) ); } }; - executorService = MoreExecutors.listeningDecorator(Execs.multiThreaded(8, "test_sql_resource_%s")); walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder(), scheduler); + resourceCloser.register(walker); + } + + @AfterClass + public static void tearDownClass() throws IOException + { + resourceCloser.close(); + } + + @Before + public void setUp() + { + executorService = MoreExecutors.listeningDecorator(Execs.multiThreaded(8, "test_sql_resource_%s")); final PlannerConfig plannerConfig = PlannerConfig.builder().serializeComplexValues(false).build(); final DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema( @@ -178,8 +181,6 @@ public class SqlStatementTest @After public void tearDown() throws Exception { - walker.close(); - walker = null; executorService.shutdownNow(); executorService.awaitTermination(2, TimeUnit.SECONDS); } @@ -222,7 +223,8 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.foo", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); DirectStatement stmt = sqlStatementFactory.directStatement(sqlReq); ResultSet resultSet = stmt.plan(); assertTrue(resultSet.runnable()); @@ -243,7 +245,8 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.foo", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); DirectStatement stmt = sqlStatementFactory.directStatement(sqlReq); stmt.plan(); try { @@ -260,7 +263,8 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.foo", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); DirectStatement stmt = sqlStatementFactory.directStatement(sqlReq); ResultSet resultSet = stmt.plan(); resultSet.run(); @@ -278,15 +282,20 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "SELECT COUNT(*) AS cnt, 'foo' AS", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); DirectStatement stmt = sqlStatementFactory.directStatement(sqlReq); try { stmt.execute(); fail(); } - catch (SqlPlanningException e) { - // Expected - assertEquals(PlanningError.SQL_PARSE_ERROR.getErrorCode(), e.getErrorCode()); + catch (DruidException e) { + MatcherAssert.assertThat( + e, + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageContains("Received an unexpected token [AS ]") + ); } } @@ -295,15 +304,20 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.bogus", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); DirectStatement stmt = sqlStatementFactory.directStatement(sqlReq); try { stmt.execute(); fail(); } - catch (SqlPlanningException e) { - // Expected - assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), e.getErrorCode()); + catch (DruidException e) { + MatcherAssert.assertThat( + e, + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageContains("Object 'bogus' not found within 'druid'") + ); } } @@ -312,7 +326,8 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "select count(*) from forbiddenDatasource", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); DirectStatement stmt = sqlStatementFactory.directStatement(sqlReq); try { stmt.execute(); @@ -336,7 +351,7 @@ public class SqlStatementTest false, null, null - ); + ); } @Test @@ -345,7 +360,7 @@ public class SqlStatementTest HttpStatement stmt = sqlStatementFactory.httpStatement( makeQuery("SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.foo"), request(true) - ); + ); List results = stmt.execute().getResults().toList(); assertEquals(1, results.size()); assertEquals(6L, results.get(0)[0]); @@ -358,14 +373,18 @@ public class SqlStatementTest HttpStatement stmt = sqlStatementFactory.httpStatement( makeQuery("SELECT COUNT(*) AS cnt, 'foo' AS"), request(true) - ); + ); try { stmt.execute(); fail(); } - catch (SqlPlanningException e) { - // Expected - assertEquals(PlanningError.SQL_PARSE_ERROR.getErrorCode(), e.getErrorCode()); + catch (DruidException e) { + MatcherAssert.assertThat( + e, + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageContains("Received an unexpected token [AS ]") + ); } } @@ -375,14 +394,18 @@ public class SqlStatementTest HttpStatement stmt = sqlStatementFactory.httpStatement( makeQuery("SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.bogus"), request(true) - ); + ); try { stmt.execute(); fail(); } - catch (SqlPlanningException e) { - // Expected - assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), e.getErrorCode()); + catch (DruidException e) { + MatcherAssert.assertThat( + e, + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageContains("Object 'bogus' not found within 'druid'") + ); } } @@ -392,7 +415,7 @@ public class SqlStatementTest HttpStatement stmt = sqlStatementFactory.httpStatement( makeQuery("select count(*) from forbiddenDatasource"), request(false) - ); + ); try { stmt.execute(); fail(); @@ -410,7 +433,8 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.foo", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); PreparedStatement stmt = sqlStatementFactory.preparedStatement(sqlReq); PrepareResult prepareResult = stmt.prepare(); @@ -440,15 +464,20 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "SELECT COUNT(*) AS cnt, 'foo' AS", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); PreparedStatement stmt = sqlStatementFactory.preparedStatement(sqlReq); try { stmt.prepare(); fail(); } - catch (SqlPlanningException e) { - // Expected - assertEquals(PlanningError.SQL_PARSE_ERROR.getErrorCode(), e.getErrorCode()); + catch (DruidException e) { + MatcherAssert.assertThat( + e, + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageContains("Received an unexpected token [AS ]") + ); } } @@ -457,15 +486,20 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.bogus", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); PreparedStatement stmt = sqlStatementFactory.preparedStatement(sqlReq); try { stmt.prepare(); fail(); } - catch (SqlPlanningException e) { - // Expected - assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), e.getErrorCode()); + catch (DruidException e) { + MatcherAssert.assertThat( + e, + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageContains("Object 'bogus' not found within 'druid'") + ); } } @@ -474,7 +508,8 @@ public class SqlStatementTest { SqlQueryPlus sqlReq = queryPlus( "select count(*) from forbiddenDatasource", - CalciteTests.REGULAR_USER_AUTH_RESULT); + CalciteTests.REGULAR_USER_AUTH_RESULT + ); PreparedStatement stmt = sqlStatementFactory.preparedStatement(sqlReq); try { stmt.prepare(); diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java index cd036d04c3a..88a237bd42d 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -45,6 +45,7 @@ import org.apache.druid.guice.StartupInjectorBuilder; import org.apache.druid.initialization.CoreInjectorBuilder; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Yielder; @@ -96,6 +97,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -123,6 +125,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; @@ -144,21 +147,27 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase // This must match the number of Connection objects created in testTooManyStatements() AVATICA_CONFIG.maxConnections = CONNECTION_LIMIT; AVATICA_CONFIG.maxStatementsPerConnection = STATEMENT_LIMIT; + System.setProperty("user.timezone", "UTC"); } private static final String DUMMY_SQL_QUERY_ID = "dummy"; + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + private static QueryRunnerFactoryConglomerate conglomerate; + private static SpecificSegmentsQuerySegmentWalker walker; private static Closer resourceCloser; private final boolean nullNumeric = !NullHandling.replaceWithDefault(); @BeforeClass - public static void setUpClass() + public static void setUpClass() throws Exception { resourceCloser = Closer.create(); conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser); - System.setProperty("user.timezone", "UTC"); + walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder()); + resourceCloser.register(walker); } @AfterClass @@ -167,16 +176,12 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase resourceCloser.close(); } - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - @Rule public QueryLogHook queryLogHook = QueryLogHook.create(); private final PlannerConfig plannerConfig = new PlannerConfig(); private final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); private final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); - private SpecificSegmentsQuerySegmentWalker walker; private ServerWrapper server; private Connection client; private Connection clientNoTrailingSlash; @@ -249,60 +254,66 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase protected AbstractAvaticaHandler getAvaticaHandler(final DruidMeta druidMeta) { return new DruidAvaticaJsonHandler( - druidMeta, - new DruidNode("dummy", "dummy", false, 1, null, true, false), - new AvaticaMonitor() + druidMeta, + new DruidNode("dummy", "dummy", false, 1, null, true, false), + new AvaticaMonitor() ); } @Before public void setUp() throws Exception { - walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder()); final DruidSchemaCatalog rootSchema = makeRootSchema(); testRequestLogger = new TestRequestLogger(); injector = new CoreInjectorBuilder(new StartupInjectorBuilder().build()) - .addModule(binder -> { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); - binder.bind(AuthenticatorMapper.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_MAPPER); - binder.bind(AuthorizerMapper.class).toInstance(CalciteTests.TEST_AUTHORIZER_MAPPER); - binder.bind(Escalator.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR); - binder.bind(RequestLogger.class).toInstance(testRequestLogger); - binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema); - for (NamedSchema schema : rootSchema.getNamedSchemas().values()) { - Multibinder.newSetBinder(binder, NamedSchema.class).addBinding().toInstance(schema); + .addModule( + binder -> { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); + binder.bind(AuthenticatorMapper.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_MAPPER); + binder.bind(AuthorizerMapper.class).toInstance(CalciteTests.TEST_AUTHORIZER_MAPPER); + binder.bind(Escalator.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR); + binder.bind(RequestLogger.class).toInstance(testRequestLogger); + binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema); + for (NamedSchema schema : rootSchema.getNamedSchemas().values()) { + Multibinder.newSetBinder(binder, NamedSchema.class).addBinding().toInstance(schema); + } + binder.bind(QueryLifecycleFactory.class) + .toInstance(CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate)); + binder.bind(DruidOperatorTable.class).toInstance(operatorTable); + binder.bind(ExprMacroTable.class).toInstance(macroTable); + binder.bind(PlannerConfig.class).toInstance(plannerConfig); + binder.bind(String.class) + .annotatedWith(DruidSchemaName.class) + .toInstance(CalciteTests.DRUID_SCHEMA_NAME); + binder.bind(AvaticaServerConfig.class).toInstance(AVATICA_CONFIG); + binder.bind(ServiceEmitter.class).to(NoopServiceEmitter.class); + binder.bind(QuerySchedulerProvider.class).in(LazySingleton.class); + binder.bind(QueryScheduler.class) + .toProvider(QuerySchedulerProvider.class) + .in(LazySingleton.class); + binder.install(new SqlModule.SqlStatementFactoryModule()); + binder.bind(new TypeLiteral>() + { + }).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of()))); + binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of())); + binder.bind(JoinableFactoryWrapper.class).toInstance(CalciteTests.createJoinableFactoryWrapper()); + binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); } - binder.bind(QueryLifecycleFactory.class) - .toInstance(CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate)); - binder.bind(DruidOperatorTable.class).toInstance(operatorTable); - binder.bind(ExprMacroTable.class).toInstance(macroTable); - binder.bind(PlannerConfig.class).toInstance(plannerConfig); - binder.bind(String.class) - .annotatedWith(DruidSchemaName.class) - .toInstance(CalciteTests.DRUID_SCHEMA_NAME); - binder.bind(AvaticaServerConfig.class).toInstance(AVATICA_CONFIG); - binder.bind(ServiceEmitter.class).to(NoopServiceEmitter.class); - binder.bind(QuerySchedulerProvider.class).in(LazySingleton.class); - binder.bind(QueryScheduler.class) - .toProvider(QuerySchedulerProvider.class) - .in(LazySingleton.class); - binder.install(new SqlModule.SqlStatementFactoryModule()); - binder.bind(new TypeLiteral>(){}).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of()))); - binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of())); - binder.bind(JoinableFactoryWrapper.class).toInstance(CalciteTests.createJoinableFactoryWrapper()); - binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); - } - ) + ) .build(); DruidMeta druidMeta = injector.getInstance(DruidMeta.class); server = new ServerWrapper(druidMeta); client = server.getUserConnection(); superuserClient = server.getConnection(CalciteTests.TEST_SUPERUSER_NAME, "druid"); - clientNoTrailingSlash = DriverManager.getConnection(StringUtils.maybeRemoveTrailingSlash(server.url), CalciteTests.TEST_SUPERUSER_NAME, "druid"); + clientNoTrailingSlash = DriverManager.getConnection( + StringUtils.maybeRemoveTrailingSlash(server.url), + CalciteTests.TEST_SUPERUSER_NAME, + "druid" + ); final Properties propertiesLosAngeles = new Properties(); propertiesLosAngeles.setProperty("sqlTimeZone", "America/Los_Angeles"); @@ -324,8 +335,6 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase clientNoTrailingSlash = null; server = null; } - walker.close(); - walker = null; } @Test @@ -820,35 +829,48 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase } @Test(timeout = 90_000L) - public void testConcurrentQueries() throws InterruptedException, ExecutionException + public void testConcurrentQueries() { - final List> futures = new ArrayList<>(); - final ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Execs.multiThreaded(AVATICA_CONFIG.getMaxStatementsPerConnection(), "DruidAvaticaHandlerTest-%d") - ); - for (int i = 0; i < 2000; i++) { - final String query = StringUtils.format("SELECT COUNT(*) + %s AS ci FROM foo", i); - futures.add( - exec.submit(() -> { - try ( - final Statement statement = client.createStatement(); - final ResultSet resultSet = statement.executeQuery(query) - ) { - final List> rows = getRows(resultSet); - return ((Number) Iterables.getOnlyElement(rows).get("ci")).intValue(); - } - catch (SQLException e) { - throw new RuntimeException(e); - } - }) - ); - } + queryLogHook.withSkippedLog( + v -> { + final List> futures = new ArrayList<>(); + final ListeningExecutorService exec = MoreExecutors.listeningDecorator( + Execs.multiThreaded(AVATICA_CONFIG.getMaxStatementsPerConnection(), "DruidAvaticaHandlerTest-%d") + ); + for (int i = 0; i < 2000; i++) { + final String query = StringUtils.format("SELECT COUNT(*) + %s AS ci FROM foo", i); + futures.add( + exec.submit(() -> { + try ( + final Statement statement = client.createStatement(); + final ResultSet resultSet = statement.executeQuery(query) + ) { + final List> rows = getRows(resultSet); + return ((Number) Iterables.getOnlyElement(rows).get("ci")).intValue(); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + }) + ); + } - final List integers = Futures.allAsList(futures).get(); - for (int i = 0; i < 2000; i++) { - Assert.assertEquals(i + 6, (int) integers.get(i)); - } - exec.shutdown(); + final List integers; + try { + integers = Futures.allAsList(futures).get(); + } + catch (InterruptedException e) { + throw new RE(e); + } + catch (ExecutionException e) { + throw new RE(e); + } + for (int i = 0; i < 2000; i++) { + Assert.assertEquals(i + 6, (int) integers.get(i)); + } + exec.shutdown(); + } + ); } @Test @@ -1246,7 +1268,8 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase @Test public void testParameterBinding() throws SQLException { - try (PreparedStatement statement = client.prepareStatement("SELECT COUNT(*) AS cnt FROM druid.foo WHERE dim1 = ? OR dim1 = ?")) { + try (PreparedStatement statement = client.prepareStatement( + "SELECT COUNT(*) AS cnt FROM druid.foo WHERE dim1 = ? OR dim1 = ?")) { statement.setString(1, "abc"); statement.setString(2, "def"); final ResultSet resultSet = statement.executeQuery(); @@ -1264,7 +1287,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase public void testSysTableParameterBindingRegularUser() throws SQLException { try (PreparedStatement statement = - client.prepareStatement("SELECT COUNT(*) AS cnt FROM sys.servers WHERE servers.host = ?")) { + client.prepareStatement("SELECT COUNT(*) AS cnt FROM sys.servers WHERE servers.host = ?")) { statement.setString(1, "dummy"); Assert.assertThrows( @@ -1279,7 +1302,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase public void testSysTableParameterBindingSuperUser() throws SQLException { try (PreparedStatement statement = - superuserClient.prepareStatement("SELECT COUNT(*) AS cnt FROM sys.servers WHERE servers.host = ?")) { + superuserClient.prepareStatement("SELECT COUNT(*) AS cnt FROM sys.servers WHERE servers.host = ?")) { statement.setString(1, "dummy"); Assert.assertEquals( ImmutableList.of( @@ -1294,7 +1317,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase public void testExecuteMany() throws SQLException { try (PreparedStatement statement = - superuserClient.prepareStatement("SELECT COUNT(*) AS cnt FROM sys.servers WHERE servers.host = ?")) { + superuserClient.prepareStatement("SELECT COUNT(*) AS cnt FROM sys.servers WHERE servers.host = ?")) { statement.setString(1, "dummy"); Assert.assertEquals( ImmutableList.of( @@ -1586,7 +1609,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase { final String query = "SELECT * FROM " + CalciteTests.FORBIDDEN_DATASOURCE; final String expectedError = "Error 2 (00002) : Error while executing SQL \"" + - query + "\": Remote driver error: " + Access.DEFAULT_ERROR_MESSAGE; + query + "\": Remote driver error: " + Access.DEFAULT_ERROR_MESSAGE; try (Statement statement = client.createStatement()) { statement.executeQuery(query); } @@ -1624,10 +1647,11 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase } /** - * Test the async aspect of the Avatica implementation. The fetch of the - * first batch takes 3 seconds (due to a sleep). However, the client will - * wait only 1 second. So, we should get ~3 empty batches before we get - * the first batch with rows. + * Test the async aspect of the Avatica implementation. Uses a countdown latches to provide + * deterministic asynchronous behavior of not having results ready for the first 3 fetches. + *

+ * We set the fetch timeout to a small 1ms value because we want the test to complete fast and + * are ensuring the proper happens-before relationships with latches instead of time. */ @Test public void testAsync() throws Exception @@ -1636,24 +1660,57 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase config.maxConnections = CONNECTION_LIMIT; config.maxStatementsPerConnection = STATEMENT_LIMIT; config.maxRowsPerFrame = 2; - config.fetchTimeoutMs = 1000; + config.fetchTimeoutMs = 1; final List frames = new ArrayList<>(); final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame"); + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch resultsLatch = new CountDownLatch(1); DruidMeta druidMeta = new DruidMeta( makeStatementFactory(), config, new ErrorHandler(new ServerConfig()), exec, injector.getInstance(AuthenticatorMapper.class).getAuthenticatorChain(), - new ResultFetcherFactory(config.fetchTimeoutMs) { + new ResultFetcherFactory(config.fetchTimeoutMs) + { + + @Override + public int fetchTimeoutMs() + { + // We override fetchTimeoutMs because the constructor here is enforcing a minimum timeout of 1s, so we + // have to workaround the constructor code by overriding this method. Luckily the internal field is + // not actually being referenced internally and is instead routing through this method. In a future + // refactoring of this code, we should move such enforcement onto the configuration layer and now + // squirreled away inside a constructor. + return config.fetchTimeoutMs; + } + @Override public ResultFetcher newFetcher( final int limit, final Yielder yielder ) { - return new TestResultFetcher(limit, yielder); + return new ResultFetcher(limit, yielder) + { + @Override + public Meta.Frame call() + { + try { + if (offset() == 0) { + startLatch.await(); + } + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + + final Meta.Frame retVal = super.call(); + resultsLatch.countDown(); + return retVal; + } + }; } } ) @@ -1665,6 +1722,15 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase final int fetchMaxRowCount ) throws NoSuchStatementException, MissingResultsException { + if (frames.size() == 3) { + startLatch.countDown(); + try { + resultsLatch.await(); + } + catch (InterruptedException e) { + throw new RE(e); + } + } Frame frame = super.fetch(statement, offset, fetchMaxRowCount); frames.add(frame); return frame; @@ -1679,10 +1745,14 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase "SELECT dim1 FROM druid.foo")) { List> rows = getRows(resultSet); Assert.assertEquals(6, rows.size()); - Assert.assertTrue(frames.size() > 3); + Assert.assertEquals(6, frames.size()); // 3 empty frames and then 3 frames of 2 rows each - // There should be at least one empty frame due to timeout Assert.assertFalse(frames.get(0).rows.iterator().hasNext()); + Assert.assertFalse(frames.get(1).rows.iterator().hasNext()); + Assert.assertFalse(frames.get(2).rows.iterator().hasNext()); + Assert.assertTrue(frames.get(3).rows.iterator().hasNext()); + Assert.assertTrue(frames.get(4).rows.iterator().hasNext()); + Assert.assertTrue(frames.get(5).rows.iterator().hasNext()); } } diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java index 618665c25f6..505bfd98e87 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java @@ -53,6 +53,7 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -72,20 +73,23 @@ public class DruidStatementTest extends CalciteTestBase private static String SELECT_STAR_FROM_FOO = "SELECT * FROM druid.foo"; - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @Rule public QueryLogHook queryLogHook = QueryLogHook.create(); + private static SpecificSegmentsQuerySegmentWalker walker; private static QueryRunnerFactoryConglomerate conglomerate; private static Closer resourceCloser; @BeforeClass - public static void setUpClass() + public static void setUpClass() throws Exception { resourceCloser = Closer.create(); conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser); + walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder()); + resourceCloser.register(walker); } @AfterClass @@ -94,13 +98,11 @@ public class DruidStatementTest extends CalciteTestBase resourceCloser.close(); } - private SpecificSegmentsQuerySegmentWalker walker; private SqlStatementFactory sqlStatementFactory; @Before - public void setUp() throws Exception + public void setUp() { - walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder()); final PlannerConfig plannerConfig = new PlannerConfig(); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); @@ -127,10 +129,9 @@ public class DruidStatementTest extends CalciteTestBase } @After - public void tearDown() throws Exception + public void tearDown() { - walker.close(); - walker = null; + } //----------------------------------------------------------------- diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 2a0930912b4..428e1d82004 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -26,10 +26,11 @@ import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.inject.Injector; -import org.apache.calcite.plan.RelOptPlanner; import org.apache.commons.text.StringEscapeUtils; import org.apache.druid.annotations.UsedByJUnitParamsRunner; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.hll.VersionOneHyperLogLogCollector; import org.apache.druid.java.util.common.DateTimes; @@ -102,6 +103,7 @@ import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSuppl import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardPlannerComponentSupplier; import org.apache.druid.sql.calcite.view.ViewManager; import org.apache.druid.sql.http.SqlParameter; +import org.hamcrest.MatcherAssert; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; @@ -487,6 +489,16 @@ public class BaseCalciteQueryTest extends CalciteTestBase queryFramework = null; } + protected static DruidExceptionMatcher invalidSqlIs(String s) + { + return DruidExceptionMatcher.invalidSqlInput().expectMessageIs(s); + } + + protected static DruidExceptionMatcher invalidSqlContains(String s) + { + return DruidExceptionMatcher.invalidSqlInput().expectMessageContains(s); + } + @Rule public QueryLogHook getQueryLogHook() { @@ -633,23 +645,25 @@ public class BaseCalciteQueryTest extends CalciteTestBase public void assertQueryIsUnplannable(final PlannerConfig plannerConfig, final String sql, String expectedError) { - Exception e = null; try { testQuery(plannerConfig, sql, CalciteTests.REGULAR_USER_AUTH_RESULT, ImmutableList.of(), ImmutableList.of()); } - catch (Exception e1) { - e = e1; + catch (DruidException e) { + MatcherAssert.assertThat( + e, + new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general") + .expectMessageIs( + StringUtils.format( + "Query planning failed for unknown reason, our best guess is this [%s]", + expectedError + ) + ) + ); } - - if (!(e instanceof RelOptPlanner.CannotPlanException)) { - log.error(e, "Expected CannotPlanException for query: %s", sql); + catch (Exception e) { + log.error(e, "Expected DruidException for query: %s", sql); Assert.fail(sql); } - Assert.assertEquals( - sql, - StringUtils.format("Query not supported. %s SQL was: %s", expectedError, sql), - e.getMessage() - ); } /** @@ -986,7 +1000,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase return getSqlStatementFactory( plannerConfig, new AuthConfig() - ); + ); } /** @@ -1028,6 +1042,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase /** * Override not just the outer query context, but also the contexts of all subqueries. + * * @return */ public static Query recursivelyClearContext(final Query query, ObjectMapper queryJsonMapper) @@ -1150,7 +1165,10 @@ public class BaseCalciteQueryTest extends CalciteTestBase output.put(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD, timestampResultField); try { - output.put(GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_GRANULARITY, queryFramework().queryJsonMapper().writeValueAsString(granularity)); + output.put( + GroupByQuery.CTX_TIMESTAMP_RESULT_FIELD_GRANULARITY, + queryFramework().queryJsonMapper().writeValueAsString(granularity) + ); } catch (JsonProcessingException e) { throw new RuntimeException(e); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java index 3a1ac2db9aa..577f46a2e99 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java @@ -54,7 +54,6 @@ import java.util.Map; @RunWith(JUnitParamsRunner.class) public class CalciteCorrelatedQueryTest extends BaseCalciteQueryTest { - @Test @Parameters(source = QueryContextForJoinProvider.class) public void testCorrelatedSubquery(Map queryContext) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java index 22855033990..071c8ae04d3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java @@ -41,7 +41,6 @@ import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.Query; @@ -457,7 +456,7 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest @Nonnull public Set getTypes() { - throw new UOE("This inputSource does not support input source based security"); + throw new CalciteIngestDmlTestException("getTypes()"); } @JsonProperty @@ -509,4 +508,12 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest return Objects.hash(files); } } + + static class CalciteIngestDmlTestException extends RuntimeException + { + public CalciteIngestDmlTestException(String message) + { + super(message); + } + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index aeabf5241a0..7fb52843b4e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -26,6 +26,8 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -41,16 +43,15 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinType; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.ForbiddenException; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.calcite.external.ExternalDataSource; import org.apache.druid.sql.calcite.external.Externals; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.planner.Calcites; -import org.apache.druid.sql.calcite.planner.IngestHandler; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.util.CalciteTests; import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; @@ -199,7 +200,11 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("INSERT INTO \"in/valid\" SELECT dim1, dim2 FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class, "INSERT dataSource cannot contain the '/' character.") + .expectValidationError( + DruidExceptionMatcher.invalidInput().expectMessageIs( + "Invalid value for field [table]: Value [in/valid] cannot contain '/'." + ) + ) .verify(); } @@ -208,7 +213,9 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("INSERT INTO dst (foo, bar) SELECT dim1, dim2 FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class, "INSERT with a target column list is not supported.") + .expectValidationError( + invalidSqlIs("Operation [INSERT] cannot be run with a target column list, given [dst (`foo`, `bar`)]") + ) .verify(); } @@ -217,7 +224,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("UPSERT INTO dst SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class, "UPSERT is not supported.") + .expectValidationError(invalidSqlIs("UPSERT is not supported.")) .verify(); } @@ -229,8 +236,8 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest testIngestionQuery() .sql("INSERT INTO dst SELECT * FROM INFORMATION_SCHEMA.COLUMNS PARTITIONED BY ALL TIME") .expectValidationError( - SqlPlanningException.class, - "Cannot query table INFORMATION_SCHEMA.COLUMNS with SQL engine 'ingestion-test'." + DruidException.class, + "Cannot query table(s) [INFORMATION_SCHEMA.COLUMNS] with SQL engine [ingestion-test]" ) .verify(); } @@ -240,10 +247,9 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("INSERT INTO INFORMATION_SCHEMA.COLUMNS SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError( - SqlPlanningException.class, - "Cannot INSERT into INFORMATION_SCHEMA.COLUMNS because it is not a Druid datasource." - ) + .expectValidationError(invalidSqlIs( + "Table [INFORMATION_SCHEMA.COLUMNS] does not support operation [INSERT] because it is not a Druid datasource" + )) .verify(); } @@ -253,8 +259,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest testIngestionQuery() .sql("INSERT INTO view.aview SELECT * FROM foo PARTITIONED BY ALL TIME") .expectValidationError( - SqlPlanningException.class, - "Cannot INSERT into view.aview because it is not a Druid datasource." + invalidSqlIs("Table [view.aview] does not support operation [INSERT] because it is not a Druid datasource") ) .verify(); } @@ -282,10 +287,9 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("INSERT INTO nonexistent.dst SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError( - SqlPlanningException.class, - "Cannot INSERT into nonexistent.dst because it is not a Druid datasource." - ) + .expectValidationError(invalidSqlIs( + "Table [nonexistent.dst] does not support operation [INSERT] because it is not a Druid datasource" + )) .verify(); } @@ -366,11 +370,12 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest throw new RuntimeException(e); } testIngestionQuery() - .sql("INSERT INTO dst SELECT * FROM %s\n" + - " (x VARCHAR, y VARCHAR, z BIGINT)\n" + - "PARTITIONED BY ALL TIME", - extern - ) + .sql( + "INSERT INTO dst SELECT * FROM %s\n" + + " (x VARCHAR, y VARCHAR, z BIGINT)\n" + + "PARTITIONED BY ALL TIME", + extern + ) .authentication(CalciteTests.SUPER_USER_AUTH_RESULT) .expectTarget("dst", externalDataSource.getSignature()) .expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION) @@ -557,9 +562,8 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .expectLogicalPlanFrom("insertFromExternal") .expectValidationError( CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo( - "org.apache.druid.java.util.common.UOE: This inputSource does not support input source based security")) + CoreMatchers.instanceOf(CalciteIngestDmlTestException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("getTypes()")) ) ) .verify(); @@ -763,8 +767,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest testIngestionQuery() .sql(sql) .expectValidationError( - SqlPlanningException.class, - "[`dim1` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order." + invalidSqlIs("Invalid CLUSTERED BY clause [`dim1` DESC]: cannot sort in descending order.") ) .verify(); } @@ -819,10 +822,9 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest + "SELECT __time, FLOOR(m1) as floor_m1, dim1, CEIL(m2) as ceil_m2 FROM foo " + "CLUSTERED BY 2, dim1 DESC, CEIL(m2)" ) - .expectValidationError( - SqlPlanningException.class, - "CLUSTERED BY found before PARTITIONED BY. In Druid, the CLUSTERED BY clause must follow the PARTITIONED BY clause" - ) + .expectValidationError(invalidSqlIs( + "CLUSTERED BY found before PARTITIONED BY, CLUSTERED BY must come after the PARTITIONED BY clause" + )) .verify(); } @@ -901,11 +903,10 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest ); Assert.fail("Exception should be thrown"); } - catch (SqlPlanningException e) { - Assert.assertEquals( - "Cannot have ORDER BY on an INSERT statement, use CLUSTERED BY instead.", - e.getMessage() - ); + catch (DruidException e) { + MatcherAssert.assertThat(e, invalidSqlIs( + "Cannot use an ORDER BY clause on a Query of type [INSERT], use CLUSTERED BY instead" + )); } didTest = true; } @@ -913,7 +914,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest @Test public void testInsertWithPartitionedByContainingInvalidGranularity() { - // Throws a ValidationException, which gets converted to a SqlPlanningException before throwing to end user + // Throws a ValidationException, which gets converted to a DruidException before throwing to end user try { testQuery( "INSERT INTO dst SELECT * FROM foo PARTITIONED BY 'invalid_granularity'", @@ -922,11 +923,13 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest ); Assert.fail("Exception should be thrown"); } - catch (SqlPlanningException e) { - Assert.assertEquals( - "Encountered 'invalid_granularity' after PARTITIONED BY. Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR function or TIME_FLOOR function", - e.getMessage() - ); + catch (DruidException e) { + MatcherAssert.assertThat( + e, + invalidSqlIs( + "Invalid granularity ['invalid_granularity'] after PARTITIONED BY. " + + "Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR() or TIME_FLOOR()" + )); } didTest = true; } @@ -945,10 +948,10 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest ); Assert.fail("Exception should be thrown"); } - catch (SqlPlanningException e) { - Assert.assertEquals( - "Cannot have ORDER BY on an INSERT statement, use CLUSTERED BY instead.", - e.getMessage() + catch (DruidException e) { + MatcherAssert.assertThat( + e, + invalidSqlIs("Cannot use an ORDER BY clause on a Query of type [INSERT], use CLUSTERED BY instead") ); } finally { @@ -959,8 +962,8 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest @Test public void testInsertWithoutPartitionedBy() { - SqlPlanningException e = Assert.assertThrows( - SqlPlanningException.class, + DruidException e = Assert.assertThrows( + DruidException.class, () -> testQuery( StringUtils.format("INSERT INTO dst SELECT * FROM %s", externSql(externalDataSource)), @@ -968,7 +971,11 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest ImmutableList.of() ) ); - Assert.assertEquals("INSERT statements must specify PARTITIONED BY clause explicitly", e.getMessage()); + + MatcherAssert.assertThat( + e, + invalidSqlIs("Operation [INSERT] requires a PARTITIONED BY to be explicitly defined, but none was found.") + ); didTest = true; } @@ -1192,7 +1199,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest { assertQueryIsUnplannable( "INSERT INTO druid.dst SELECT dim2, dim1, m1 FROM foo2 UNION SELECT dim1, dim2, m1 FROM foo PARTITIONED BY ALL TIME", - "Possible error: SQL requires 'UNION' but only 'UNION ALL' is supported." + "SQL requires 'UNION' but only 'UNION ALL' is supported." ); // Not using testIngestionQuery, so must set didTest manually to satisfy the check in tearDown. @@ -1308,14 +1315,13 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest @Test public void testInsertWithInvalidSelectStatement() { + // This test fails because "count" is a reserved word and it is being used without quotes. So SQL is considering + // it a token instead of a name. It would be nice if our message was more direct telling the person that they + // used a reserved word instead of making them know that a "token" means Calcite is seeing a reserved word. But, + // that's an improvement for another day. testIngestionQuery() .sql("INSERT INTO t SELECT channel, added as count FROM foo PARTITIONED BY ALL") // count is a keyword - .expectValidationError( - CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Encountered \"as count\"")) - ) - ) + .expectValidationError(invalidSqlContains("Received an unexpected token [as count]")) .verify(); } @@ -1324,10 +1330,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("INSERT INTO t SELECT dim1, dim2 || '-lol' FROM foo PARTITIONED BY ALL") - .expectValidationError( - SqlPlanningException.class, - IngestHandler.UNNAMED_INGESTION_COLUMN_ERROR - ) + .expectValidationError(invalidSqlContains("Insertion requires columns to be named")) .verify(); } @@ -1336,10 +1339,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("INSERT INTO t SELECT __time, dim1 AS EXPR$0 FROM foo PARTITIONED BY ALL") - .expectValidationError( - SqlPlanningException.class, - IngestHandler.UNNAMED_INGESTION_COLUMN_ERROR - ) + .expectValidationError(invalidSqlContains("Insertion requires columns to be named")) .verify(); } @@ -1350,10 +1350,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .sql("INSERT INTO test " + "SELECT __time, * FROM " + "(SELECT __time, LOWER(dim1) FROM foo) PARTITIONED BY ALL TIME") - .expectValidationError( - SqlPlanningException.class, - IngestHandler.UNNAMED_INGESTION_COLUMN_ERROR - ) + .expectValidationError(invalidSqlContains("Insertion requires columns to be named")) .verify(); } @@ -1364,11 +1361,11 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .sql("insert into foo1 select __time, dim1 FROM foo partitioned by time_floor(__time, 'PT2H')") .expectValidationError( CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), + CoreMatchers.instanceOf(DruidException.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "The granularity specified in PARTITIONED BY is not supported. " - + "Please use an equivalent of these granularities: second, minute, five_minute, ten_minute, " - + "fifteen_minute, thirty_minute, hour, six_hour, eight_hour, day, week, month, quarter, year, all.")) + "The granularity specified in PARTITIONED BY [`time_floor`(`__time`, 'PT2H')] is not supported. " + + "Valid options: [second, minute, five_minute, ten_minute, fifteen_minute, thirty_minute, hour, " + + "six_hour, eight_hour, day, week, month, quarter, year, all]")) ) ) .verify(); @@ -1391,7 +1388,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest ) .expectValidationError( CoreMatchers.allOf( - CoreMatchers.instanceOf(SqlPlanningException.class), + CoreMatchers.instanceOf(DruidException.class), ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( "EXTERN function with __time column can be used when __time column is of type long")) ) @@ -1409,8 +1406,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest .context(context) .sql("INSERT INTO dst SELECT * FROM foo PARTITIONED BY ALL TIME") .expectValidationError( - SqlPlanningException.class, - "sqlOuterLimit cannot be provided with INSERT." + invalidSqlIs("Context parameter [sqlOuterLimit] cannot be provided on operator [INSERT]") ) .verify(); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index 1f45aebc8b5..337926d462a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -25,6 +25,8 @@ import com.google.common.collect.ImmutableSet; import junitparams.JUnitParamsRunner; import junitparams.Parameters; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; @@ -89,8 +91,8 @@ import org.apache.druid.server.security.Access; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.planner.PlannerConfig; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.hamcrest.MatcherAssert; import org.joda.time.DateTimeZone; import org.joda.time.Period; import org.junit.Assert; @@ -1032,7 +1034,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @Test @Parameters(source = QueryContextForJoinProvider.class) public void testInnerJoinTableLookupLookupWithFilterWithOuterLimit(Map queryContext) @@ -1483,16 +1484,31 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @Test(expected = UnsupportedSQLQueryException.class) + @Test @Parameters(source = QueryContextForJoinProvider.class) public void testTimeColumnAggregationsOnLookups(Map queryContext) { - testQuery( - "SELECT k, LATEST(v) v FROM lookup.lookyloo GROUP BY k", - queryContext, - ImmutableList.of(), - ImmutableList.of() - ); + try { + testQuery( + "SELECT k, LATEST(v) v FROM lookup.lookyloo GROUP BY k", + queryContext, + ImmutableList.of(), + ImmutableList.of() + ); + Assert.fail("Expected exception to be thrown."); + } + catch (DruidException e) { + MatcherAssert.assertThat( + e, + new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general") + .expectMessageIs( + "Query planning failed for unknown reason, our best guess is this " + + "[LATEST and EARLIEST aggregators implicitly depend on the __time column, " + + "but the table queried doesn't contain a __time column. " + + "Please use LATEST_BY or EARLIEST_BY and specify the column explicitly.]" + ) + ); + } } @Test @@ -3342,7 +3358,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest { assertQueryIsUnplannable( "SELECT t1.dim1 from foo as t1 LEFT JOIN foo as t2 on t1.dim1 = '10.1'", - "Possible error: SQL is resulting in a join that has unsupported operand types." + "SQL is resulting in a join that has unsupported operand types." ); } @@ -3498,6 +3514,9 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .context(queryContext) .build(); + boolean isJoinFilterRewriteEnabled = queryContext.getOrDefault(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, true) + .toString() + .equals("true"); testQuery( "SELECT dim1, l1.k\n" + "FROM foo\n" @@ -3505,7 +3524,16 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest + "WHERE l1.k IS NOT NULL\n", queryContext, ImmutableList.of(NullHandling.sqlCompatible() ? nullCompatibleModePlan : nonNullCompatibleModePlan), - ImmutableList.of(new Object[]{"abc", "abc"}) + NullHandling.sqlCompatible() || !isJoinFilterRewriteEnabled + ? ImmutableList.of(new Object[]{"abc", "abc"}) + : ImmutableList.of( + new Object[]{"10.1", ""}, + // this result is incorrect. TODO : fix this result when the JoinFilterAnalyzer bug is fixed + new Object[]{"2", ""}, + new Object[]{"1", ""}, + new Object[]{"def", ""}, + new Object[]{"abc", "abc"} + ) ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java index 2311372e75e..d016f179532 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java @@ -22,6 +22,7 @@ package org.apache.druid.sql.calcite; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExpressionProcessing; @@ -43,7 +44,6 @@ import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.virtual.ListFilteredVirtualColumn; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; import org.junit.Test; @@ -801,7 +801,11 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_ordinal(\"dim3\",2)", ColumnType.STRING)) + .setVirtualColumns(expressionVirtualColumn( + "v0", + "array_ordinal(\"dim3\",2)", + ColumnType.STRING + )) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING) @@ -1797,7 +1801,7 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest testQueryThrows( "SELECT MV_TO_ARRAY(dim3,dim3) FROM druid.numfoo", exception -> { - exception.expect(SqlPlanningException.class); + exception.expect(DruidException.class); exception.expectMessage("Invalid number of arguments to function"); } ); @@ -1809,7 +1813,7 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest testQueryThrows( "SELECT MV_TO_ARRAY() FROM druid.numfoo", exception -> { - exception.expect(SqlPlanningException.class); + exception.expect(DruidException.class); exception.expectMessage("Invalid number of arguments to function"); } ); @@ -2008,8 +2012,7 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest "SELECT COALESCE(dim3, 'other') FROM druid.numfoo " + "WHERE MV_OVERLAP(COALESCE(dim3, ARRAY['other']), ARRAY['a', 'b', 'other']) LIMIT 5", e -> { - e.expect(SqlPlanningException.class); - e.expectMessage("Illegal mixing of types in CASE or COALESCE statement"); + e.expect(invalidSqlContains("Illegal mixing of types in CASE or COALESCE statement")); } ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 007bb4926d7..83861485960 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -32,6 +32,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.HumanReadableBytes; @@ -68,7 +69,6 @@ import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.virtual.NestedFieldVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.sql.calcite.filtration.Filtration; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; @@ -4172,9 +4172,11 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest + "SUM(cnt) " + "FROM druid.nested GROUP BY 1", (expected) -> { - expected.expect(UnsupportedSQLQueryException.class); - expected.expectMessage( - "Cannot use [JSON_VALUE_VARCHAR]: [Bad format, '.array.[1]' is not a valid JSONPath path: must start with '$']"); + expected.expect( + DruidExceptionMatcher + .invalidInput() + .expectMessageIs("JSONPath [.array.[1]] is invalid, it must start with '$'") + ); } ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java index 3e1bfe62b65..2266b8d6b82 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java @@ -22,6 +22,7 @@ package org.apache.druid.sql.calcite; import com.google.common.collect.ImmutableList; import org.apache.calcite.avatica.SqlType; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -37,7 +38,6 @@ import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQuery.ResultFormat; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.http.SqlParameter; @@ -577,8 +577,9 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest @Test public void testMissingParameter() { - expectedException.expect(SqlPlanningException.class); - expectedException.expectMessage("Parameter at position [0] is not bound"); + expectedException.expect( + DruidExceptionMatcher.invalidSqlInput().expectMessageIs("No value bound for parameter (position [1])") + ); testQuery( "SELECT COUNT(*)\n" + "FROM druid.numfoo\n" @@ -592,8 +593,9 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest @Test public void testPartiallyMissingParameter() { - expectedException.expect(SqlPlanningException.class); - expectedException.expectMessage("Parameter at position [1] is not bound"); + expectedException.expect( + DruidExceptionMatcher.invalidSqlInput().expectMessageIs("No value bound for parameter (position [2])") + ); testQuery( "SELECT COUNT(*)\n" + "FROM druid.numfoo\n" @@ -610,8 +612,9 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest List params = new ArrayList<>(); params.add(null); params.add(new SqlParameter(SqlType.INTEGER, 1)); - expectedException.expect(SqlPlanningException.class); - expectedException.expectMessage("Parameter at position [0] is not bound"); + expectedException.expect( + DruidExceptionMatcher.invalidSqlInput().expectMessageIs("No value bound for parameter (position [1])") + ); testQuery( "SELECT 1 + ?, dim1 FROM foo LIMIT ?", ImmutableList.of(), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 0e7d377489d..792c25a5e1d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -23,9 +23,9 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.runtime.CalciteContextException; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.Intervals; @@ -110,8 +110,6 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinType; -import org.apache.druid.sql.SqlPlanningException; -import org.apache.druid.sql.SqlPlanningException.PlanningError; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.planner.Calcites; @@ -372,8 +370,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest public void testCannotInsertWithNativeEngine() { notMsqCompatible(); - final SqlPlanningException e = Assert.assertThrows( - SqlPlanningException.class, + final DruidException e = Assert.assertThrows( + DruidException.class, () -> testQuery( "INSERT INTO dst SELECT * FROM foo PARTITIONED BY ALL", ImmutableList.of(), @@ -383,9 +381,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest MatcherAssert.assertThat( e, - ThrowableMessageMatcher.hasMessage( - CoreMatchers.equalTo("Cannot execute INSERT with SQL engine 'native'.") - ) + invalidSqlIs("INSERT operations are not supported by requested SQL engine [native], consider using MSQ.") ); } @@ -393,8 +389,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest public void testCannotReplaceWithNativeEngine() { notMsqCompatible(); - final SqlPlanningException e = Assert.assertThrows( - SqlPlanningException.class, + final DruidException e = Assert.assertThrows( + DruidException.class, () -> testQuery( "REPLACE INTO dst OVERWRITE ALL SELECT * FROM foo PARTITIONED BY ALL", ImmutableList.of(), @@ -404,9 +400,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest MatcherAssert.assertThat( e, - ThrowableMessageMatcher.hasMessage( - CoreMatchers.equalTo("Cannot execute REPLACE with SQL engine 'native'.") - ) + invalidSqlIs("REPLACE operations are not supported by the requested SQL engine [native]. Consider using MSQ.") ); } @@ -814,7 +808,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testEarliestByInvalidTimestamp() { - expectedException.expect(SqlPlanningException.class); + expectedException.expect(DruidException.class); expectedException.expectMessage("Cannot apply 'EARLIEST_BY' to arguments of type 'EARLIEST_BY(, )"); testQuery( @@ -827,8 +821,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testLatestByInvalidTimestamp() { - expectedException.expect(SqlPlanningException.class); - expectedException.expectMessage("Cannot apply 'LATEST_BY' to arguments of type 'LATEST_BY(, )"); + expectedException.expect( + invalidSqlContains("Cannot apply 'LATEST_BY' to arguments of type 'LATEST_BY(, )") + ); testQuery( "SELECT LATEST_BY(m1, l1) FROM druid.numfoo", @@ -1067,22 +1062,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest public void testStringLatestGroupByWithAlwaysFalseCondition() { testQuery( - "SELECT LATEST(dim4, 10),dim2 FROM numfoo WHERE (dim1 = 'something' AND dim1 IN( 'something else') ) GROUP BY dim2", + "SELECT LATEST(dim4, 10), dim2 FROM numfoo WHERE (dim1 = 'something' AND dim1 IN('something else')) GROUP BY dim2", ImmutableList.of( Druids.newScanQueryBuilder() - .dataSource(InlineDataSource.fromIterable( - ImmutableList.of(), - RowSignature.builder() - .add("EXPR$0", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .build() - )) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("EXPR$0", "dim2") - .context(QUERY_CONTEXT_DEFAULT) - .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) - .build() + .dataSource(InlineDataSource.fromIterable( + ImmutableList.of(), + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("dim2", ColumnType.STRING) + .build() + )) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("EXPR$0", "dim2") + .context(QUERY_CONTEXT_DEFAULT) + .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .build() ), ImmutableList.of() ); @@ -1092,22 +1087,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest public void testStringLatestByGroupByWithAlwaysFalseCondition() { testQuery( - "SELECT LATEST_BY(dim4, __time, 10),dim2 FROM numfoo WHERE (dim1 = 'something' AND dim1 IN( 'something else') ) GROUP BY dim2", + "SELECT LATEST_BY(dim4, __time, 10), dim2 FROM numfoo WHERE (dim1 = 'something' AND dim1 IN('something else')) GROUP BY dim2", ImmutableList.of( Druids.newScanQueryBuilder() - .dataSource(InlineDataSource.fromIterable( - ImmutableList.of(), - RowSignature.builder() - .add("EXPR$0", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .build() - )) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("EXPR$0", "dim2") - .context(QUERY_CONTEXT_DEFAULT) - .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) - .build() + .dataSource(InlineDataSource.fromIterable( + ImmutableList.of(), + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("dim2", ColumnType.STRING) + .build() + )) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("EXPR$0", "dim2") + .context(QUERY_CONTEXT_DEFAULT) + .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .build() ), ImmutableList.of() ); @@ -2903,12 +2898,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); Assert.fail("query execution should fail"); } - catch (SqlPlanningException e) { - Assert.assertTrue( - e.getMessage().contains("Column count mismatch in UNION ALL") - ); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), e.getErrorCode()); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorClass(), e.getErrorClass()); + catch (DruidException e) { + MatcherAssert.assertThat(e, invalidSqlIs("Column count mismatch in UNION ALL (line [3], column [42])")); } } @@ -2971,22 +2962,24 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + "FROM (SELECT dim3, dim2, m1 FROM foo2 UNION ALL SELECT dim3, dim2, m1 FROM foo)\n" + "WHERE dim2 = 'a' OR dim2 = 'en'\n" + "GROUP BY 1, 2", - "Possible error: SQL requires union between inputs that are not simple table scans and involve a " + - "filter or aliasing. Or column types of tables being unioned are not of same type."); + "SQL requires union between inputs that are not simple table scans and involve a " + + "filter or aliasing. Or column types of tables being unioned are not of same type." + ); } @Test public void testUnionAllTablesWhenMappingIsRequired() { // Cannot plan this UNION ALL operation, because the column swap would require generating a subquery. + assertQueryIsUnplannable( "SELECT\n" + "c, COUNT(*)\n" + "FROM (SELECT dim1 AS c, m1 FROM foo UNION ALL SELECT dim2 AS c, m1 FROM numfoo)\n" + "WHERE c = 'a' OR c = 'def'\n" + "GROUP BY 1", - "Possible error: SQL requires union between two tables " + - "and column names queried for each table are different Left: [dim1], Right: [dim2]." + "SQL requires union between two tables " + + "and column names queried for each table are different Left: [dim1], Right: [dim2]." ); } @@ -2996,7 +2989,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest // Cannot plan this UNION operation assertQueryIsUnplannable( "SELECT dim2, dim1, m1 FROM foo2 UNION SELECT dim1, dim2, m1 FROM foo", - "Possible error: SQL requires 'UNION' but only 'UNION ALL' is supported." + "SQL requires 'UNION' but only 'UNION ALL' is supported." ); } @@ -3010,8 +3003,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + "FROM (SELECT dim1 AS c, m1 FROM foo UNION ALL SELECT cnt AS c, m1 FROM numfoo)\n" + "WHERE c = 'a' OR c = 'def'\n" + "GROUP BY 1", - "Possible error: SQL requires union between inputs that are not simple table scans and involve " + - "a filter or aliasing. Or column types of tables being unioned are not of same type." + "SQL requires union between inputs that are not simple table scans and involve " + + "a filter or aliasing. Or column types of tables being unioned are not of same type." ); } @@ -3111,7 +3104,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + "FROM (SELECT dim1, dim2, m1 FROM foo UNION ALL SELECT dim2, dim1, m1 FROM foo)\n" + "WHERE dim2 = 'a' OR dim2 = 'def'\n" + "GROUP BY 1, 2", - "Possible error: SQL requires union between two tables and column names queried for each table are different Left: [dim1, dim2, m1], Right: [dim2, dim1, m1]." + "SQL requires union between two tables and column names queried for each table are different Left: [dim1, dim2, m1], Right: [dim2, dim1, m1]." ); } @@ -3174,12 +3167,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); Assert.fail("query execution should fail"); } - catch (SqlPlanningException e) { - Assert.assertTrue( - e.getMessage().contains("Column count mismatch in UNION ALL") - ); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), e.getErrorCode()); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorClass(), e.getErrorClass()); + catch (DruidException e) { + MatcherAssert.assertThat(e, invalidSqlIs("Column count mismatch in UNION ALL (line [3], column [45])")); } } @@ -3198,12 +3187,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); Assert.fail("query execution should fail"); } - catch (SqlPlanningException e) { - Assert.assertTrue( - e.getMessage().contains("Column count mismatch in UNION ALL") - ); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), e.getErrorCode()); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorClass(), e.getErrorClass()); + catch (DruidException e) { + MatcherAssert.assertThat(e, invalidSqlIs("Column count mismatch in UNION ALL (line [3], column [45])")); } } @@ -3222,12 +3207,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); Assert.fail("query execution should fail"); } - catch (SqlPlanningException e) { - Assert.assertTrue( - e.getMessage().contains("Column count mismatch in UNION ALL") - ); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), e.getErrorCode()); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorClass(), e.getErrorClass()); + catch (DruidException e) { + MatcherAssert.assertThat(e, invalidSqlIs("Column count mismatch in UNION ALL (line [3], column [70])")); } } @@ -5634,17 +5615,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest final Map queries = ImmutableMap.of( // SELECT query with order by non-__time. "SELECT dim1 FROM druid.foo ORDER BY dim1", - "Possible error: SQL query requires order by non-time column [dim1 ASC], which is not supported.", + "SQL query requires order by non-time column [[dim1 ASC]], which is not supported.", // JOIN condition with not-equals (<>). "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + "FROM foo INNER JOIN lookup.lookyloo l ON foo.dim2 <> l.k", - "Possible error: SQL requires a join with 'NOT_EQUALS' condition that is not supported.", + "SQL requires a join with 'NOT_EQUALS' condition that is not supported.", // JOIN condition with a function of both sides. "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + "FROM foo INNER JOIN lookup.lookyloo l ON CHARACTER_LENGTH(foo.dim2 || l.k) > 3\n", - "Possible error: SQL requires a join with 'GREATER_THAN' condition that is not supported." + "SQL requires a join with 'GREATER_THAN' condition that is not supported." ); for (final Map.Entry queryErrorPair : queries.entrySet()) { @@ -5702,7 +5683,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest assertQueryIsUnplannable( PLANNER_CONFIG_NO_HLL, "SELECT dim2, COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo GROUP BY dim2", - "Possible error: SQL requires a join with 'IS_NOT_DISTINCT_FROM' condition that is not supported." + "SQL requires a join with 'IS_NOT_DISTINCT_FROM' condition that is not supported." ); } @@ -5713,7 +5694,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest assertQueryIsUnplannable( PLANNER_CONFIG_NO_HLL, "SELECT COUNT(distinct unique_dim1) FROM druid.foo", - "Possible error: SQL requires a group-by on a column of type COMPLEX that is unsupported." + "SQL requires a group-by on a column of type COMPLEX that is unsupported." ); } @@ -5756,7 +5737,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .build() ), ImmutableList.of( - new Object[]{"[\"AQAAAEAAAA==\",\"AQAAAQAAAAHNBA==\",\"AQAAAQAAAAOzAg==\",\"AQAAAQAAAAFREA==\",\"AQAAAQAAAACyEA==\",\"AQAAAQAAAAEkAQ==\"]"} + new Object[]{ + "[\"AQAAAEAAAA==\",\"AQAAAQAAAAHNBA==\",\"AQAAAQAAAAOzAg==\",\"AQAAAQAAAAFREA==\",\"AQAAAQAAAACyEA==\",\"AQAAAQAAAAEkAQ==\"]" + } ) ); } @@ -5768,12 +5751,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest testQuery("SELECT STRING_AGG(unique_dim1, ',') FROM druid.foo", ImmutableList.of(), ImmutableList.of()); Assert.fail("query execution should fail"); } - catch (SqlPlanningException e) { - Assert.assertTrue( - e.getMessage().contains("Cannot use STRING_AGG on complex inputs COMPLEX") + catch (DruidException e) { + MatcherAssert.assertThat( + e, + invalidSqlIs("Aggregation [STRING_AGG] does not support type [COMPLEX], column [foo.unique_dim1]") ); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), e.getErrorCode()); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorClass(), e.getErrorClass()); } } @@ -5909,11 +5891,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest "SELECT COUNT(*) FROM druid.foo " + "WHERE TIME_IN_INTERVAL(__time, dim1)", expected -> { - expected.expect(CoreMatchers.instanceOf(SqlPlanningException.class)); - expected.expect(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "From line 1, column 38 to line 1, column 67: " - + "Cannot apply 'TIME_IN_INTERVAL' to arguments of type 'TIME_IN_INTERVAL(, )'. " - + "Supported form(s): 'TIME_IN_INTERVAL(, )'"))); + expected.expect( + invalidSqlIs( + "Cannot apply 'TIME_IN_INTERVAL' to arguments of type " + + "'TIME_IN_INTERVAL(, )'. Supported form(s): " + + "'TIME_IN_INTERVAL(, )' (line [1], column [38])" + ) + ); } ); } @@ -6050,11 +6034,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest { // Strings are implicitly cast to timestamps. Test an invalid string. // This error message isn't ideal but it is at least better than silently ignoring the problem. - assertQueryIsUnplannable( - "SELECT COUNT(*) FROM druid.foo\n" - + "WHERE __time >= 'z2000-01-01 00:00:00' AND __time < '2001-01-01 00:00:00'\n", - "Possible error: Illegal TIMESTAMP constant: CAST('z2000-01-01 00:00:00'):TIMESTAMP(3) NOT NULL" - ); + String sql = "SELECT COUNT(*) FROM druid.foo\n" + + "WHERE __time >= 'z2000-01-01 00:00:00' AND __time < '2001-01-01 00:00:00'\n"; + try { + testBuilder().sql(sql).run(); + } + catch (DruidException e) { + MatcherAssert.assertThat( + e, + invalidSqlIs("Illegal TIMESTAMP constant [CAST('z2000-01-01 00:00:00'):TIMESTAMP(3) NOT NULL]") + ); + } + catch (Exception e) { + log.error(e, "Expected DruidException for query: %s", sql); + Assert.fail(sql); + } } @Test @@ -7548,7 +7542,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest { notMsqCompatible(); expectedException.expect(UOE.class); - expectedException.expectMessage("The number of values in the IN clause for [dim6] in query exceeds configured maxNumericFilter limit of [2] for INs. Cast [3] values of IN clause to String"); + expectedException.expectMessage( + "The number of values in the IN clause for [dim6] in query exceeds configured maxNumericFilter limit of [2] for INs. Cast [3] values of IN clause to String"); testQuery( PLANNER_CONFIG_MAX_NUMERIC_IN_FILTER, @@ -11301,12 +11296,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest testQuery("SELECT TIME_EXTRACT(__time) FROM druid.foo", ImmutableList.of(), ImmutableList.of()); Assert.fail("query execution should fail"); } - catch (SqlPlanningException e) { - Assert.assertTrue( - e.getMessage().contains("Invalid number of arguments to function 'TIME_EXTRACT'. Was expecting 2 arguments") + catch (DruidException e) { + MatcherAssert.assertThat( + e, + invalidSqlIs( + "Invalid number of arguments to function 'TIME_EXTRACT'. Was expecting 2 arguments (line [1], column [8])" + ) ); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), e.getErrorCode()); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorClass(), e.getErrorClass()); } } @@ -12828,8 +12824,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest //Since adding a zero period does not change the timestamp, just compare the stamp with the orignal TestDataBuilder.ROWS1.stream() - .map(row -> new Object[]{row.getTimestampFromEpoch()}) - .collect(Collectors.toList()) + .map(row -> new Object[]{row.getTimestampFromEpoch()}) + .collect(Collectors.toList()) ); } @@ -12857,8 +12853,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest //Since adding a zero period does not change the timestamp, just compare the stamp with the orignal TestDataBuilder.ROWS1.stream() - .map(row -> new Object[]{row.getTimestampFromEpoch()}) - .collect(Collectors.toList()) + .map(row -> new Object[]{row.getTimestampFromEpoch()}) + .collect(Collectors.toList()) ); } @@ -12888,8 +12884,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest //Since adding a zero period does not change the timestamp, just compare the stamp with the orignal TestDataBuilder.ROWS1.stream() - .map(row -> new Object[]{row.getTimestampFromEpoch()}) - .collect(Collectors.toList()) + .map(row -> new Object[]{row.getTimestampFromEpoch()}) + .collect(Collectors.toList()) ); } @@ -12926,8 +12922,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest // verify if query results match the given TestDataBuilder.ROWS1.stream() - .map(r -> new Object[]{periodGranularity.increment(r.getTimestamp()).getMillis()}) - .collect(Collectors.toList()) + .map(r -> new Object[]{periodGranularity.increment(r.getTimestamp()).getMillis()}) + .collect(Collectors.toList()) ); // @@ -12955,8 +12951,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest // verify if query results match the given // "cnt" for each row is 1 TestDataBuilder.ROWS1.stream() - .map(row -> new Object[]{periodGranularity.increment(row.getTimestamp()).getMillis()}) - .collect(Collectors.toList()) + .map(row -> new Object[]{periodGranularity.increment(row.getTimestamp()).getMillis()}) + .collect(Collectors.toList()) ); } @@ -13961,7 +13957,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } - @Test(expected = RelOptPlanner.CannotPlanException.class) + @Test(expected = DruidException.class) public void testStringAggExpressionNonConstantSeparator() { testQuery( @@ -14110,7 +14106,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testHumanReadableFormatFunctionExceptionWithWrongNumberType() { - this.expectedException.expect(SqlPlanningException.class); + this.expectedException.expect(DruidException.class); this.expectedException.expectMessage("Supported form(s): HUMAN_READABLE_BINARY_BYTE_FORMAT(Number, [Precision])"); testQuery( "SELECT HUMAN_READABLE_BINARY_BYTE_FORMAT('45678')", @@ -14122,7 +14118,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testHumanReadableFormatFunctionWithWrongPrecisionType() { - this.expectedException.expect(SqlPlanningException.class); + this.expectedException.expect(DruidException.class); this.expectedException.expectMessage("Supported form(s): HUMAN_READABLE_BINARY_BYTE_FORMAT(Number, [Precision])"); testQuery( "SELECT HUMAN_READABLE_BINARY_BYTE_FORMAT(45678, '2')", @@ -14134,7 +14130,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testHumanReadableFormatFunctionWithInvalidNumberOfArguments() { - this.expectedException.expect(SqlPlanningException.class); + this.expectedException.expect(DruidException.class); /* * frankly speaking, the exception message thrown here is a little bit confusing @@ -14545,21 +14541,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest testQuery( "SELECT COMPLEX_DECODE_BASE64('hyperUnique',PARSE_JSON(TO_JSON_STRING(unique_dim1))) from druid.foo LIMIT 10", ImmutableList.of( - Druids.newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("v0") - .virtualColumns( - expressionVirtualColumn( - "v0", - "complex_decode_base64('hyperUnique',parse_json(to_json_string(\"unique_dim1\")))", - ColumnType.ofComplex("hyperUnique") - ) - ) - .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) - .limit(10) - .build() + Druids.newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("v0") + .virtualColumns( + expressionVirtualColumn( + "v0", + "complex_decode_base64('hyperUnique',parse_json(to_json_string(\"unique_dim1\")))", + ColumnType.ofComplex("hyperUnique") + ) + ) + .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .limit(10) + .build() ), ImmutableList.of( new Object[]{"\"AQAAAEAAAA==\""}, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index 0c1f016600d..d7ba655c1ef 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -22,6 +22,8 @@ package org.apache.druid.sql.calcite; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.jackson.JacksonUtils; @@ -33,7 +35,6 @@ import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.server.security.ForbiddenException; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.calcite.external.Externals; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; @@ -41,6 +42,7 @@ import org.apache.druid.sql.calcite.parser.DruidSqlParserUtils; import org.apache.druid.sql.calcite.parser.DruidSqlReplace; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; @@ -49,8 +51,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; -import static org.junit.Assert.assertEquals; - public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { private static final Map REPLACE_ALL_TIME_CHUNKS = ImmutableMap.of( @@ -218,10 +218,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE WHERE __time LIKE '20__-02-01' SELECT * FROM foo PARTITIONED BY MONTH") - .expectValidationError( - SqlPlanningException.class, - "Unsupported operation in OVERWRITE WHERE clause: LIKE" - ) + .expectValidationError(invalidSqlIs( + "Invalid OVERWRITE WHERE clause [`__time` LIKE '20__-02-01']: Unsupported operation [LIKE] in OVERWRITE WHERE clause." + )) .verify(); } @@ -230,10 +229,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE WHERE TRUE SELECT * FROM foo PARTITIONED BY MONTH") - .expectValidationError( - SqlPlanningException.class, - "Invalid OVERWRITE WHERE clause" - ) + .expectValidationError(invalidSqlIs( + "Invalid OVERWRITE WHERE clause [TRUE]: expected clause including AND, OR, NOT, >, <, >=, <= OR BETWEEN operators" + )) .verify(); } @@ -242,10 +240,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE WHERE dim1 > TIMESTAMP '2000-01-05 00:00:00' SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError( - SqlPlanningException.class, - "Only __time column is supported in OVERWRITE WHERE clause" - ) + .expectValidationError(invalidSqlIs( + "OVERWRITE WHERE clause only supports filtering on the __time column, got [947030400000 < dim1 as numeric]" + )) .verify(); } @@ -255,7 +252,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE ALL SELECT * FROM foo ORDER BY dim1 PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class, "Cannot have ORDER BY on a REPLACE statement, use CLUSTERED BY instead.") + .expectValidationError(invalidSqlIs( + "Cannot use an ORDER BY clause on a Query of type [REPLACE], use CLUSTERED BY instead" + )) .verify(); } @@ -265,8 +264,10 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE WHERE __time >= TIMESTAMP '2000-01-05 00:00:00' AND __time <= TIMESTAMP '2000-01-06 00:00:00' SELECT * FROM foo PARTITIONED BY MONTH") .expectValidationError( - SqlPlanningException.class, - "OVERWRITE WHERE clause contains an interval [2000-01-05T00:00:00.000Z/2000-01-06T00:00:00.001Z] which is not aligned with PARTITIONED BY granularity {type=period, period=P1M, timeZone=UTC, origin=null}" + invalidSqlIs( + "OVERWRITE WHERE clause identified interval [2000-01-05T00:00:00.000Z/2000-01-06T00:00:00.001Z] " + + "which is not aligned with PARTITIONED BY granularity [{type=period, period=P1M, timeZone=UTC, origin=null}]" + ) ) .verify(); } @@ -276,10 +277,10 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE WHERE __time >= TIMESTAMP '2000-01-05 00:00:00' AND __time <= TIMESTAMP '2000-02-05 00:00:00' SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError( - SqlPlanningException.class, - "OVERWRITE WHERE clause contains an interval [2000-01-05T00:00:00.000Z/2000-02-05T00:00:00.001Z] which is not aligned with PARTITIONED BY granularity AllGranularity" - ) + .expectValidationError(invalidSqlIs( + "OVERWRITE WHERE clause identified interval [2000-01-05T00:00:00.000Z/2000-02-05T00:00:00.001Z] " + + "which is not aligned with PARTITIONED BY granularity [AllGranularity]" + )) .verify(); } @@ -290,10 +291,10 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest .sql("REPLACE INTO dst OVERWRITE WHERE " + "__time < TIMESTAMP '2000-01-01' AND __time > TIMESTAMP '2000-01-01' " + "SELECT * FROM foo PARTITIONED BY MONTH") - .expectValidationError( - SqlPlanningException.class, - "Intervals for replace are empty" - ) + .expectValidationError(invalidSqlIs( + "The OVERWRITE WHERE clause [(__time as numeric < 946684800000 && 946684800000 < __time as numeric)] " + + "produced no time intervals, are the bounds overly restrictive?" + )) .verify(); } @@ -302,7 +303,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE WHERE __time >= TIMESTAMP '2000-01-INVALID0:00' AND __time <= TIMESTAMP '2000-02-05 00:00:00' SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class) + .expectValidationError(DruidException.class) .verify(); } @@ -311,7 +312,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class) + .expectValidationError(DruidException.class) .verify(); } @@ -381,7 +382,11 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO \"in/valid\" OVERWRITE ALL SELECT dim1, dim2 FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class, "REPLACE dataSource cannot contain the '/' character.") + .expectValidationError( + DruidExceptionMatcher + .invalidInput() + .expectMessageIs("Invalid value for field [table]: Value [in/valid] cannot contain '/'.") + ) .verify(); } @@ -390,7 +395,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst (foo, bar) OVERWRITE ALL SELECT dim1, dim2 FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class, "REPLACE with a target column list is not supported.") + .expectValidationError( + invalidSqlIs("Operation [REPLACE] cannot be run with a target column list, given [dst (`foo`, `bar`)]") + ) .verify(); } @@ -399,7 +406,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE ALL SELECT __time, FLOOR(m1) as floor_m1, dim1 FROM foo") - .expectValidationError(SqlPlanningException.class, "REPLACE statements must specify PARTITIONED BY clause explicitly") + .expectValidationError(invalidSqlIs( + "Operation [REPLACE] requires a PARTITIONED BY to be explicitly defined, but none was found." + )) .verify(); } @@ -408,7 +417,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE ALL SELECT __time, FLOOR(m1) as floor_m1, dim1 FROM foo CLUSTERED BY dim1") - .expectValidationError(SqlPlanningException.class, "CLUSTERED BY found before PARTITIONED BY. In Druid, the CLUSTERED BY clause must follow the PARTITIONED BY clause") + .expectValidationError(invalidSqlIs( + "CLUSTERED BY found before PARTITIONED BY, CLUSTERED BY must come after the PARTITIONED BY clause" + )) .verify(); } @@ -417,7 +428,10 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class, "Missing time chunk information in OVERWRITE clause for REPLACE. Use OVERWRITE WHERE <__time based condition> or OVERWRITE ALL to overwrite the entire table.") + .expectValidationError(invalidSqlIs( + "Missing time chunk information in OVERWRITE clause for REPLACE. " + + "Use OVERWRITE WHERE <__time based condition> or OVERWRITE ALL to overwrite the entire table." + )) .verify(); } @@ -426,7 +440,10 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO dst OVERWRITE SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class, "Missing time chunk information in OVERWRITE clause for REPLACE. Use OVERWRITE WHERE <__time based condition> or OVERWRITE ALL to overwrite the entire table.") + .expectValidationError(invalidSqlIs( + "Missing time chunk information in OVERWRITE clause for REPLACE. " + + "Use OVERWRITE WHERE <__time based condition> or OVERWRITE ALL to overwrite the entire table." + )) .verify(); } @@ -435,10 +452,10 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO INFORMATION_SCHEMA.COLUMNS OVERWRITE ALL SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError( - SqlPlanningException.class, - "Cannot REPLACE into INFORMATION_SCHEMA.COLUMNS because it is not a Druid datasource." - ) + .expectValidationError(invalidSqlIs( + "Table [INFORMATION_SCHEMA.COLUMNS] does not support operation [REPLACE]" + + " because it is not a Druid datasource" + )) .verify(); } @@ -447,10 +464,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO view.aview OVERWRITE ALL SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError( - SqlPlanningException.class, - "Cannot REPLACE into view.aview because it is not a Druid datasource." - ) + .expectValidationError(invalidSqlIs( + "Table [view.aview] does not support operation [REPLACE] because it is not a Druid datasource" + )) .verify(); } @@ -477,10 +493,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest { testIngestionQuery() .sql("REPLACE INTO nonexistent.dst OVERWRITE ALL SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError( - SqlPlanningException.class, - "Cannot REPLACE into nonexistent.dst because it is not a Druid datasource." - ) + .expectValidationError(invalidSqlIs( + "Table [nonexistent.dst] does not support operation [REPLACE] because it is not a Druid datasource" + )) .verify(); } @@ -576,7 +591,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest @Test public void testReplaceWithPartitionedByContainingInvalidGranularity() { - // Throws a ValidationException, which gets converted to a SqlPlanningException before throwing to end user + // Throws a ValidationException, which gets converted to a DruidException before throwing to end user try { testQuery( "REPLACE INTO dst OVERWRITE ALL SELECT * FROM foo PARTITIONED BY 'invalid_granularity'", @@ -585,10 +600,13 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest ); Assert.fail("Exception should be thrown"); } - catch (SqlPlanningException e) { - assertEquals( - "Encountered 'invalid_granularity' after PARTITIONED BY. Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR function or TIME_FLOOR function", - e.getMessage() + catch (DruidException e) { + MatcherAssert.assertThat( + e, + invalidSqlIs( + "Invalid granularity ['invalid_granularity'] after PARTITIONED BY. " + + "Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR() or TIME_FLOOR()" + ) ); } didTest = true; @@ -779,8 +797,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest testIngestionQuery() .sql(sql) .expectValidationError( - SqlPlanningException.class, - "[`dim1` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order." + invalidSqlIs("Invalid CLUSTERED BY clause [`dim1` DESC]: cannot sort in descending order.") ) .verify(); } @@ -909,7 +926,9 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest testIngestionQuery() .context(context) .sql("REPLACE INTO dst OVERWRITE ALL SELECT * FROM foo PARTITIONED BY ALL TIME") - .expectValidationError(SqlPlanningException.class, "sqlOuterLimit cannot be provided with REPLACE.") + .expectValidationError(DruidExceptionMatcher.invalidInput().expectMessageIs( + "Context parameter [sqlOuterLimit] cannot be provided on operator [REPLACE]" + )) .verify(); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java index 54d8e856af8..cc60a27acdd 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java @@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite; import com.google.common.collect.ImmutableList; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -43,7 +44,6 @@ import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.util.CalciteTests; @@ -311,8 +311,10 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest @Test public void testSelectConstantExpressionEquivalentToNaN() { - expectedException.expectMessage( - "'(log10(0) - log10(0))' evaluates to 'NaN' that is not supported in SQL. You can either cast the expression as BIGINT ('CAST((log10(0) - log10(0)) as BIGINT)') or VARCHAR ('CAST((log10(0) - log10(0)) as VARCHAR)') or change the expression itself"); + expectedException.expect(invalidSqlIs( + "Expression [(log10(0) - log10(0))] evaluates to an unsupported value [NaN], " + + "expected something that can be a Double. Consider casting with 'CAST( AS BIGINT)'" + )); testQuery( "SELECT log10(0) - log10(0), dim1 FROM foo LIMIT 1", ImmutableList.of(), @@ -323,8 +325,10 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest @Test public void testSelectConstantExpressionEquivalentToInfinity() { - expectedException.expectMessage( - "'log10(0)' evaluates to '-Infinity' that is not supported in SQL. You can either cast the expression as BIGINT ('CAST(log10(0) as BIGINT)') or VARCHAR ('CAST(log10(0) as VARCHAR)') or change the expression itself"); + expectedException.expect(invalidSqlIs( + "Expression [log10(0)] evaluates to an unsupported value [-Infinity], " + + "expected something that can be a Double. Consider casting with 'CAST( AS BIGINT)'" + )); testQuery( "SELECT log10(0), dim1 FROM foo LIMIT 1", ImmutableList.of(), @@ -964,7 +968,7 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest testQueryThrows( "SELECT CURRENT_TIMESTAMP(4)", expectedException -> { - expectedException.expect(SqlPlanningException.class); + expectedException.expect(DruidException.class); expectedException.expectMessage( "Argument to function 'CURRENT_TIMESTAMP' must be a valid precision between '0' and '3'" ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java index 34c8e490414..963e1e0b23b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java @@ -618,7 +618,9 @@ public class QueryTestRunner public QueryTestRunner(QueryTestBuilder builder) { QueryTestConfig config = builder.config; - Assume.assumeTrue(!config.isRunningMSQ() || builder.msqCompatible); + if (config.isRunningMSQ()) { + Assume.assumeTrue(builder.msqCompatible); + } if (builder.expectedResultsVerifier == null && builder.expectedResults != null) { builder.expectedResultsVerifier = config.defaultResultsVerifier( builder.expectedResults, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java index d7579679275..157a9e27175 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java @@ -22,7 +22,6 @@ package org.apache.druid.sql.calcite; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import org.apache.calcite.tools.ValidationException; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; @@ -62,7 +61,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import javax.annotation.Nullable; - import java.io.IOException; import java.util.List; import java.util.Map; @@ -181,13 +179,12 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe } @Test - public void testQuery() throws ValidationException + public void testQuery() { sanityTestVectorizedSqlQueries(PLANNER_FACTORY, query); } public static void sanityTestVectorizedSqlQueries(PlannerFactory plannerFactory, String query) - throws ValidationException { final Map vector = ImmutableMap.of( QueryContexts.VECTORIZE_KEY, "force", diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java index 1f295ea3587..01f0544e156 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java @@ -31,10 +31,12 @@ import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlPostfixOperator; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.calcite.tools.ValidationException; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.sql.calcite.expression.builtin.TimeFloorOperatorConversion; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; import org.junit.experimental.runners.Enclosed; @@ -132,12 +134,8 @@ public class DruidSqlParserUtilsTest public void testEmptyClusteredByColumnsValid() { final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); - try { - DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs); - } - catch (ValidationException e) { - Assert.fail("Did not expect an exception" + e.getMessage()); - } + + DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs); } /** @@ -151,12 +149,7 @@ public class DruidSqlParserUtilsTest clusteredByArgs.add(new SqlIdentifier("DIM2 ASC", SqlParserPos.ZERO)); clusteredByArgs.add(SqlLiteral.createExactNumeric("3", SqlParserPos.ZERO)); - try { - DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs); - } - catch (ValidationException e) { - Assert.fail("Did not expect an exception" + e.getMessage()); - } + DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs); } /** @@ -179,14 +172,10 @@ public class DruidSqlParserUtilsTest ); clusteredByArgs.add(sqlBasicCall); - ValidationException e = Assert.assertThrows( - ValidationException.class, - () -> DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs) - ); - Assert.assertEquals( - "[`DIM4` DESC] is invalid. CLUSTERED BY columns cannot be sorted in descending order.", - e.getMessage() - ); + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageIs("Invalid CLUSTERED BY clause [`DIM4` DESC]: cannot sort in descending order.") + .assertThrowsAndMatches(() -> DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs)); } } @@ -199,13 +188,18 @@ public class DruidSqlParserUtilsTest public void testConvertSqlNodeToGranularityWithIncorrectNode() { SqlNode sqlNode = SqlLiteral.createCharString("day", SqlParserPos.ZERO); - ParseException e = Assert.assertThrows( - ParseException.class, + DruidException e = Assert.assertThrows( + DruidException.class, () -> DruidSqlParserUtils.convertSqlNodeToGranularityThrowingParseExceptions(sqlNode) ); - Assert.assertEquals( - "Encountered 'day' after PARTITIONED BY. Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR function or TIME_FLOOR function", - e.getMessage() + MatcherAssert.assertThat( + e, + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageIs( + "Invalid granularity ['day'] after PARTITIONED BY. " + + "Expected HOUR, DAY, MONTH, YEAR, ALL TIME, FLOOR() or TIME_FLOOR()" + ) ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRuleTest.java index de7a005852d..73054f506a2 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRuleTest.java @@ -29,10 +29,10 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.DateString; import org.apache.calcite.util.TimeString; import org.apache.calcite.util.TimestampString; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.sql.calcite.planner.DruidTypeSystem; import org.apache.druid.sql.calcite.planner.PlannerContext; -import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.junit.Assert; @@ -177,8 +177,14 @@ public class DruidLogicalValuesRuleTest new TimestampString("2021-04-01 16:54:31"), 0 ); - expectedException.expect(UnsupportedSQLQueryException.class); - expectedException.expectMessage("TIMESTAMP_WITH_LOCAL_TIME_ZONE type is not supported"); + expectedException.expect( + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageIs( + "Cannot handle literal [2021-04-01 16:54:31:TIMESTAMP_WITH_LOCAL_TIME_ZONE(0)] " + + "of unsupported type [TIMESTAMP_WITH_LOCAL_TIME_ZONE]." + ) + ); DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT); } @@ -186,8 +192,11 @@ public class DruidLogicalValuesRuleTest public void testGetValueFromTimeLiteral() { RexLiteral literal = REX_BUILDER.makeTimeLiteral(new TimeString("16:54:31"), 0); - expectedException.expect(UnsupportedSQLQueryException.class); - expectedException.expectMessage("TIME type is not supported"); + expectedException.expect( + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageIs("Cannot handle literal [16:54:31] of unsupported type [TIME].") + ); DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT); } @@ -195,8 +204,14 @@ public class DruidLogicalValuesRuleTest public void testGetValueFromTimeWithLocalTimeZoneLiteral() { RexLiteral literal = REX_BUILDER.makeTimeWithLocalTimeZoneLiteral(new TimeString("16:54:31"), 0); - expectedException.expect(UnsupportedSQLQueryException.class); - expectedException.expectMessage("TIME_WITH_LOCAL_TIME_ZONE type is not supported"); + expectedException.expect( + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageIs( + "Cannot handle literal [16:54:31:TIME_WITH_LOCAL_TIME_ZONE(0)] " + + "of unsupported type [TIME_WITH_LOCAL_TIME_ZONE]." + ) + ); DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryLogHook.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryLogHook.java index 967926681fb..c95e2e60920 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryLogHook.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryLogHook.java @@ -31,6 +31,7 @@ import org.junit.runner.Description; import org.junit.runners.model.Statement; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import java.util.function.Supplier; @@ -43,6 +44,7 @@ public class QueryLogHook implements TestRule private final Supplier objectMapperSupplier; private final List> recordedQueries = Lists.newCopyOnWriteArrayList(); + private final AtomicBoolean skipLog = new AtomicBoolean(false); public QueryLogHook(final Supplier objectMapperSupplier) { @@ -69,6 +71,17 @@ public class QueryLogHook implements TestRule return ImmutableList.copyOf(recordedQueries); } + public void withSkippedLog(Consumer consumer) + { + try { + skipLog.set(true); + consumer.accept(null); + } + finally { + skipLog.set(false); + } + } + @Override public Statement apply(final Statement base, final Description description) { @@ -80,6 +93,10 @@ public class QueryLogHook implements TestRule clearRecordedQueries(); final Consumer function = query -> { + if (skipLog.get()) { + return; + } + try { recordedQueries.add((Query) query); log.info( diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index f825423ba9c..7dbc5ce6931 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -34,6 +34,10 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.exception.AllowedRegexErrorResponseTransformStrategy; import org.apache.druid.common.exception.ErrorResponseTransformStrategy; import org.apache.druid.common.guava.SettableSupplier; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.error.ErrorResponse; +import org.apache.druid.error.QueryExceptionCompat; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.NonnullPair; @@ -81,7 +85,6 @@ import org.apache.druid.sql.DirectStatement; import org.apache.druid.sql.HttpStatement; import org.apache.druid.sql.PreparedStatement; import org.apache.druid.sql.SqlLifecycleManager; -import org.apache.druid.sql.SqlPlanningException.PlanningError; import org.apache.druid.sql.SqlQueryPlus; import org.apache.druid.sql.SqlStatementFactory; import org.apache.druid.sql.SqlToolbox; @@ -104,8 +107,11 @@ import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -133,11 +139,13 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; +@SuppressWarnings("ALL") public class SqlResourceTest extends CalciteTestBase { public static final DruidNode DUMMY_DRUID_NODE = new DruidNode("dummy", "dummy", false, 1, null, true, false); @@ -159,13 +167,17 @@ public class SqlResourceTest extends CalciteTestBase private static final List EXPECTED_SQL_TYPES_FOR_RESULT_FORMAT_TESTS = Arrays.asList("TIMESTAMP", "VARCHAR", "VARCHAR", "VARCHAR", "BIGINT", "FLOAT", "DOUBLE", "OTHER", "VARCHAR"); + private static Closer staticCloser = Closer.create(); private static QueryRunnerFactoryConglomerate conglomerate; - private static Closer resourceCloser; - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + private static SpecificSegmentsQuerySegmentWalker walker; + private static QueryScheduler scheduler; + @Rule public QueryLogHook queryLogHook = QueryLogHook.create(); - private SpecificSegmentsQuerySegmentWalker walker; + + private Closer resourceCloser; private TestRequestLogger testRequestLogger; private SqlResource resource; private MockHttpServletRequest req; @@ -183,15 +195,13 @@ public class SqlResourceTest extends CalciteTestBase private final SettableSupplier responseContextSupplier = new SettableSupplier<>(); private Consumer onExecute = NULL_ACTION; - private Supplier schedulerBaggage = () -> null; + private static final AtomicReference> SCHEDULER_BAGGAGE = new AtomicReference<>(); - @Before - public void setUp() throws Exception + @BeforeClass + public static void setupClass() throws Exception { - resourceCloser = Closer.create(); - conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser); - - final QueryScheduler scheduler = new QueryScheduler( + conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(staticCloser); + scheduler = new QueryScheduler( 5, ManualQueryPrioritizationStrategy.INSTANCE, new HiLoQueryLaningStrategy(40), @@ -204,15 +214,29 @@ public class SqlResourceTest extends CalciteTestBase return super.run( query, new LazySequence<>(() -> { - schedulerBaggage.get(); + SCHEDULER_BAGGAGE.get().get(); return resultSequence; }) ); } }; + walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder(), scheduler); + staticCloser.register(walker); + } + + @AfterClass + public static void teardownClass() throws Exception + { + staticCloser.close(); + } + + @Before + public void setUp() throws Exception + { + SCHEDULER_BAGGAGE.set(() -> null); + resourceCloser = Closer.create(); executorService = MoreExecutors.listeningDecorator(Execs.multiThreaded(8, "test_sql_resource_%s")); - walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder(), scheduler); final PlannerConfig plannerConfig = PlannerConfig.builder().serializeComplexValues(false).build(); final DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema( @@ -320,8 +344,8 @@ public class SqlResourceTest extends CalciteTestBase @After public void tearDown() throws Exception { - walker.close(); - walker = null; + SCHEDULER_BAGGAGE.set(() -> null); + executorService.shutdownNow(); executorService.awaitTermination(2, TimeUnit.SECONDS); resourceCloser.close(); @@ -650,7 +674,7 @@ public class SqlResourceTest extends CalciteTestBase sequenceMapFnSupplier.set(errorAfterSecondRowMapFn()); final String query = "SELECT cnt FROM foo"; - final Pair response = + final Pair response = doPostRaw(new SqlQuery(query, ResultFormat.ARRAY, false, false, false, null, null), req); // Truncated response: missing final ] @@ -664,7 +688,7 @@ public class SqlResourceTest extends CalciteTestBase sequenceMapFnSupplier.set(errorAfterSecondRowMapFn()); final String query = "SELECT cnt FROM foo"; - final Pair response = + final Pair response = doPostRaw(new SqlQuery(query, ResultFormat.OBJECT, false, false, false, null, null), req); // Truncated response: missing final ] @@ -678,7 +702,7 @@ public class SqlResourceTest extends CalciteTestBase sequenceMapFnSupplier.set(errorAfterSecondRowMapFn()); final String query = "SELECT cnt FROM foo"; - final Pair response = + final Pair response = doPostRaw(new SqlQuery(query, ResultFormat.ARRAYLINES, false, false, false, null, null), req); // Truncated response: missing final LFLF @@ -692,7 +716,7 @@ public class SqlResourceTest extends CalciteTestBase sequenceMapFnSupplier.set(errorAfterSecondRowMapFn()); final String query = "SELECT cnt FROM foo"; - final Pair response = + final Pair response = doPostRaw(new SqlQuery(query, ResultFormat.OBJECTLINES, false, false, false, null, null), req); // Truncated response: missing final LFLF @@ -706,7 +730,7 @@ public class SqlResourceTest extends CalciteTestBase sequenceMapFnSupplier.set(errorAfterSecondRowMapFn()); final String query = "SELECT cnt FROM foo"; - final Pair response = + final Pair response = doPostRaw(new SqlQuery(query, ResultFormat.CSV, false, false, false, null, null), req); // Truncated response: missing final LFLF @@ -851,7 +875,7 @@ public class SqlResourceTest extends CalciteTestBase public void testArrayLinesResultFormat() throws Exception { final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2"; - final Pair pair = doPostRaw( + final Pair pair = doPostRaw( new SqlQuery(query, ResultFormat.ARRAYLINES, false, false, false, null, null) ); Assert.assertNull(pair.lhs); @@ -896,7 +920,7 @@ public class SqlResourceTest extends CalciteTestBase public void testArrayLinesResultFormatWithHeader() throws Exception { final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2"; - final Pair pair = doPostRaw( + final Pair pair = doPostRaw( new SqlQuery(query, ResultFormat.ARRAYLINES, true, true, true, null, null) ); Assert.assertNull(pair.lhs); @@ -944,7 +968,7 @@ public class SqlResourceTest extends CalciteTestBase public void testArrayLinesResultFormatWithHeader_nullColumnType() throws Exception { final String query = "SELECT (1, 2) FROM INFORMATION_SCHEMA.COLUMNS LIMIT 1"; - final Pair pair = doPostRaw( + final Pair pair = doPostRaw( new SqlQuery(query, ResultFormat.ARRAYLINES, true, true, true, null, null) ); Assert.assertNull(pair.lhs); @@ -1018,7 +1042,7 @@ public class SqlResourceTest extends CalciteTestBase public void testObjectLinesResultFormat() throws Exception { final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2"; - final Pair pair = doPostRaw( + final Pair pair = doPostRaw( new SqlQuery(query, ResultFormat.OBJECTLINES, false, false, false, null, null) ); Assert.assertNull(pair.lhs); @@ -1075,7 +1099,7 @@ public class SqlResourceTest extends CalciteTestBase public void testObjectLinesResultFormatWithMinimalHeader() throws Exception { final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2"; - final Pair pair = + final Pair pair = doPostRaw(new SqlQuery(query, ResultFormat.OBJECTLINES, true, false, false, null, null)); Assert.assertNull(pair.lhs); final String response = pair.rhs; @@ -1135,7 +1159,7 @@ public class SqlResourceTest extends CalciteTestBase public void testObjectLinesResultFormatWithFullHeader() throws Exception { final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2"; - final Pair pair = + final Pair pair = doPostRaw(new SqlQuery(query, ResultFormat.OBJECTLINES, true, true, true, null, null)); Assert.assertNull(pair.lhs); final String response = pair.rhs; @@ -1201,7 +1225,7 @@ public class SqlResourceTest extends CalciteTestBase public void testObjectLinesResultFormatWithFullHeader_nullColumnType() throws Exception { final String query = "SELECT (1, 2) FROM INFORMATION_SCHEMA.COLUMNS LIMIT 1"; - final Pair pair = + final Pair pair = doPostRaw(new SqlQuery(query, ResultFormat.OBJECTLINES, true, true, true, null, null)); Assert.assertNull(pair.lhs); final String response = pair.rhs; @@ -1230,7 +1254,7 @@ public class SqlResourceTest extends CalciteTestBase public void testCsvResultFormat() throws Exception { final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2"; - final Pair pair = doPostRaw( + final Pair pair = doPostRaw( new SqlQuery(query, ResultFormat.CSV, false, false, false, null, null) ); Assert.assertNull(pair.lhs); @@ -1252,7 +1276,7 @@ public class SqlResourceTest extends CalciteTestBase public void testCsvResultFormatWithHeaders() throws Exception { final String query = "SELECT *, CASE dim2 WHEN '' THEN dim2 END FROM foo LIMIT 2"; - final Pair pair = doPostRaw( + final Pair pair = doPostRaw( new SqlQuery(query, ResultFormat.CSV, true, true, true, null, null) ); Assert.assertNull(pair.lhs); @@ -1277,7 +1301,7 @@ public class SqlResourceTest extends CalciteTestBase public void testCsvResultFormatWithHeaders_nullColumnType() throws Exception { final String query = "SELECT (1, 2) FROM INFORMATION_SCHEMA.COLUMNS LIMIT 1"; - final Pair pair = doPostRaw( + final Pair pair = doPostRaw( new SqlQuery(query, ResultFormat.CSV, true, true, true, null, null) ); Assert.assertNull(pair.lhs); @@ -1338,12 +1362,12 @@ public class SqlResourceTest extends CalciteTestBase @Test public void testCannotParse() throws Exception { - QueryException exception = postSyncForException("FROM druid.foo", Status.BAD_REQUEST.getStatusCode()); + ErrorResponse errorResponse = postSyncForException("FROM druid.foo", Status.BAD_REQUEST.getStatusCode()); - Assert.assertNotNull(exception); - Assert.assertEquals(PlanningError.SQL_PARSE_ERROR.getErrorCode(), exception.getErrorCode()); - Assert.assertEquals(PlanningError.SQL_PARSE_ERROR.getErrorClass(), exception.getErrorClass()); - Assert.assertTrue(exception.getMessage().contains("Encountered \"FROM\" at line 1, column 1.")); + validateInvalidSqlError( + errorResponse, + "Received an unexpected token [FROM] (line [1], column [1]), acceptable options: [\"INSERT\", \"UPSERT\", " + ); checkSqlRequestLog(false); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); } @@ -1351,12 +1375,15 @@ public class SqlResourceTest extends CalciteTestBase @Test public void testCannotValidate() throws Exception { - QueryException exception = postSyncForException("SELECT dim4 FROM druid.foo", Status.BAD_REQUEST.getStatusCode()); + ErrorResponse errorResponse = postSyncForException( + "SELECT dim4 FROM druid.foo", + Status.BAD_REQUEST.getStatusCode() + ); - Assert.assertNotNull(exception); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), exception.getErrorCode()); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorClass(), exception.getErrorClass()); - Assert.assertTrue(exception.getMessage().contains("Column 'dim4' not found in any table")); + validateInvalidSqlError( + errorResponse, + "Column 'dim4' not found in any table" + ); checkSqlRequestLog(false); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); } @@ -1366,16 +1393,17 @@ public class SqlResourceTest extends CalciteTestBase { // SELECT + ORDER unsupported final SqlQuery unsupportedQuery = createSimpleQueryWithId("id", "SELECT dim1 FROM druid.foo ORDER BY dim1"); - QueryException exception = postSyncForException(unsupportedQuery, Status.BAD_REQUEST.getStatusCode()); + ErrorResponse exception = postSyncForException(unsupportedQuery, Status.BAD_REQUEST.getStatusCode()); Assert.assertTrue((Boolean) req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)); - Assert.assertNotNull(exception); - Assert.assertEquals("SQL query is unsupported", exception.getErrorCode()); - Assert.assertEquals(PlanningError.UNSUPPORTED_SQL_ERROR.getErrorClass(), exception.getErrorClass()); - Assert.assertTrue( - exception.getMessage() - .contains("Query not supported. " + - "Possible error: SQL query requires order by non-time column [dim1 ASC], which is not supported.") + + validateErrorResponse( + exception, + "general", + DruidException.Persona.ADMIN, + DruidException.Category.INVALID_INPUT, + "Query planning failed for unknown reason, our best guess is this " + + "[SQL query requires order by non-time column [[dim1 ASC]], which is not supported.]" ); checkSqlRequestLog(false); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); @@ -1390,18 +1418,14 @@ public class SqlResourceTest extends CalciteTestBase public void testCannotConvert_UnsupportedSQLQueryException() throws Exception { // max(string) unsupported - QueryException exception = postSyncForException( + ErrorResponse errorResponse = postSyncForException( "SELECT max(dim1) FROM druid.foo", Status.BAD_REQUEST.getStatusCode() ); - Assert.assertNotNull(exception); - Assert.assertEquals(PlanningError.UNSUPPORTED_SQL_ERROR.getErrorCode(), exception.getErrorCode()); - Assert.assertEquals(PlanningError.UNSUPPORTED_SQL_ERROR.getErrorClass(), exception.getErrorClass()); - Assert.assertTrue( - exception.getMessage() - .contains("Query not supported. " + - "Possible error: Max aggregation is not supported for 'STRING' type") + validateInvalidSqlError( + errorResponse, + "Aggregation [MAX] does not support type [STRING], column [v0]" ); checkSqlRequestLog(false); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); @@ -1410,7 +1434,7 @@ public class SqlResourceTest extends CalciteTestBase @Test public void testResourceLimitExceeded() throws Exception { - final QueryException exception = doPost( + final ErrorResponse errorResponse = doPost( new SqlQuery( "SELECT DISTINCT dim1 FROM foo", ResultFormat.OBJECT, @@ -1422,10 +1446,12 @@ public class SqlResourceTest extends CalciteTestBase ) ).lhs; - Assert.assertNotNull(exception); - Assert.assertEquals(exception.getErrorCode(), QueryException.RESOURCE_LIMIT_EXCEEDED_ERROR_CODE); - Assert.assertEquals(exception.getErrorClass(), ResourceLimitExceededException.class.getName()); - checkSqlRequestLog(false); + validateLegacyQueryExceptionErrorResponse( + errorResponse, + QueryException.RESOURCE_LIMIT_EXCEEDED_ERROR_CODE, + ResourceLimitExceededException.class.getName(), + "" + ); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); } @@ -1441,7 +1467,7 @@ public class SqlResourceTest extends CalciteTestBase { String errorMessage = "This will be supported in Druid 9999"; failOnExecute(errorMessage); - QueryException exception = postSyncForException( + ErrorResponse exception = postSyncForException( new SqlQuery( "SELECT ANSWER TO LIFE", ResultFormat.OBJECT, @@ -1454,9 +1480,12 @@ public class SqlResourceTest extends CalciteTestBase 501 ); - Assert.assertNotNull(exception); - Assert.assertEquals(QueryException.QUERY_UNSUPPORTED_ERROR_CODE, exception.getErrorCode()); - Assert.assertEquals(QueryUnsupportedException.class.getName(), exception.getErrorClass()); + validateLegacyQueryExceptionErrorResponse( + exception, + QueryException.QUERY_UNSUPPORTED_ERROR_CODE, + QueryUnsupportedException.class.getName(), + "" + ); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); } @@ -1537,7 +1566,7 @@ public class SqlResourceTest extends CalciteTestBase String errorMessage = "This will be supported in Druid 9999"; failOnExecute(errorMessage); - QueryException exception = postSyncForException( + ErrorResponse exception = postSyncForException( new SqlQuery( "SELECT ANSWER TO LIFE", ResultFormat.OBJECT, @@ -1550,48 +1579,38 @@ public class SqlResourceTest extends CalciteTestBase 501 ); - Assert.assertNotNull(exception); - Assert.assertNull(exception.getMessage()); - Assert.assertNull(exception.getHost()); - Assert.assertEquals(exception.getErrorCode(), QueryException.QUERY_UNSUPPORTED_ERROR_CODE); - Assert.assertNull(exception.getErrorClass()); + validateLegacyQueryExceptionErrorResponse( + exception, + QueryException.QUERY_UNSUPPORTED_ERROR_CODE, + "org.apache.druid.query.QueryUnsupportedException", + "This will be supported in Druid 9999" + ); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); } + /** + * There are various points where Calcite feels it is acceptable to throw an AssertionError when it receives bad + * input. This is unfortunate as a java.lang.Error is very clearly documented to be something that nobody should + * try to catch. But, we can editorialize all we want, we still have to deal with it. So, this test reproduces + * the AssertionError behavior by using the substr() command. At the time that this test was written, the + * SQL substr assumes a literal for the second argument. The code ends up calling `RexLiteral.intValue` on the + * argument, which ends up calling a method that fails with an AssertionError, so this should generate the + * bad behavior for us. This test is validating that our exception handling deals with this meaningfully. + * If this test starts failing, it could be indicative of us not handling the AssertionErrors well anymore, + * OR it could be indicative of this specific code path not throwing an AssertionError anymore. If we run + * into the latter case, we should seek out a new code path that generates the error from Calcite. In the best + * world, this test starts failing because Calcite has moved all of its execptions away from AssertionErrors + * and we can no longer reproduce the behavior through Calcite, in that world, we should remove our own handling + * and this test at the same time. + * + * @throws Exception + */ @Test public void testAssertionErrorThrowsErrorWithFilterResponse() throws Exception { - resource = new SqlResource( - JSON_MAPPER, - CalciteTests.TEST_AUTHORIZER_MAPPER, - sqlStatementFactory, - lifecycleManager, - new ServerConfig() - { - @Override - public boolean isShowDetailedJettyErrors() - { - return true; - } - - @Override - public ErrorResponseTransformStrategy getErrorResponseTransformStrategy() - { - return new AllowedRegexErrorResponseTransformStrategy(ImmutableList.of()); - } - }, - TEST_RESPONSE_CONTEXT_CONFIG, - DUMMY_DRUID_NODE - ); - - String errorMessage = "could not assert"; - failOnExecute(errorMessage); - onExecute = s -> { - throw new AssertionError(errorMessage); - }; - QueryException exception = postSyncForException( + ErrorResponse exception = postSyncForException( new SqlQuery( - "SELECT ANSWER TO LIFE", + "SELECT *, substr(dim2, strpos(dim2, 'hi')+2, 2) FROM foo LIMIT 2", ResultFormat.OBJECT, false, false, @@ -1599,14 +1618,15 @@ public class SqlResourceTest extends CalciteTestBase ImmutableMap.of("sqlQueryId", "id"), null ), - Status.INTERNAL_SERVER_ERROR.getStatusCode() + Status.BAD_REQUEST.getStatusCode() ); - Assert.assertNotNull(exception); - Assert.assertNull(exception.getMessage()); - Assert.assertNull(exception.getHost()); - Assert.assertEquals("Unknown exception", exception.getErrorCode()); - Assert.assertNull(exception.getErrorClass()); + MatcherAssert.assertThat( + exception.getUnderlyingException(), + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageIs("Calcite assertion violated: [not a literal: +(STRPOS($2, 'hi'), 2)]") + ); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); } @@ -1617,7 +1637,7 @@ public class SqlResourceTest extends CalciteTestBase CountDownLatch queriesScheduledLatch = new CountDownLatch(numQueries - 1); CountDownLatch runQueryLatch = new CountDownLatch(1); - schedulerBaggage = () -> { + SCHEDULER_BAGGAGE.set(() -> { queriesScheduledLatch.countDown(); try { runQueryLatch.await(); @@ -1626,7 +1646,7 @@ public class SqlResourceTest extends CalciteTestBase throw new RE(e); } return null; - }; + }); final String sqlQueryId = "tooManyRequestsTest"; @@ -1654,7 +1674,7 @@ public class SqlResourceTest extends CalciteTestBase } queriesScheduledLatch.await(); - schedulerBaggage = () -> null; + SCHEDULER_BAGGAGE.set(() -> null); futures.add(executorService.submit(() -> { try { final Response retVal = postForSyncResponse( @@ -1717,7 +1737,7 @@ public class SqlResourceTest extends CalciteTestBase sqlQueryId ); - QueryException exception = postSyncForException( + ErrorResponse exception = postSyncForException( new SqlQuery( "SELECT CAST(__time AS DATE), dim1, dim2, dim3 FROM druid.foo GROUP by __time, dim1, dim2, dim3 ORDER BY dim2 DESC", ResultFormat.OBJECT, @@ -1730,9 +1750,12 @@ public class SqlResourceTest extends CalciteTestBase 504 ); - Assert.assertNotNull(exception); - Assert.assertEquals(exception.getErrorCode(), QueryException.QUERY_TIMEOUT_ERROR_CODE); - Assert.assertEquals(exception.getErrorClass(), QueryTimeoutException.class.getName()); + validateLegacyQueryExceptionErrorResponse( + exception, + QueryException.QUERY_TIMEOUT_ERROR_CODE, + QueryTimeoutException.class.getName(), + "" + ); Assert.assertTrue(lifecycleManager.getAll(sqlQueryId).isEmpty()); } @@ -1762,8 +1785,13 @@ public class SqlResourceTest extends CalciteTestBase Response queryResponse = future.get(); assertStatusAndCommonHeaders(queryResponse, Status.INTERNAL_SERVER_ERROR.getStatusCode()); - QueryException exception = deserializeResponse(queryResponse, QueryException.class); - Assert.assertEquals("Query cancelled", exception.getErrorCode()); + ErrorResponse exception = deserializeResponse(queryResponse, ErrorResponse.class); + validateLegacyQueryExceptionErrorResponse( + exception, + "Query cancelled", + null, + "" + ); } @Test @@ -1790,8 +1818,8 @@ public class SqlResourceTest extends CalciteTestBase Response queryResponse = future.get(); assertStatusAndCommonHeaders(queryResponse, Status.INTERNAL_SERVER_ERROR.getStatusCode()); - QueryException exception = deserializeResponse(queryResponse, QueryException.class); - Assert.assertEquals("Query cancelled", exception.getErrorCode()); + ErrorResponse exception = deserializeResponse(queryResponse, ErrorResponse.class); + validateLegacyQueryExceptionErrorResponse(exception, "Query cancelled", null, ""); } @Test @@ -1854,7 +1882,7 @@ public class SqlResourceTest extends CalciteTestBase BaseQuery.SQL_QUERY_ID, sqlQueryId ); - final QueryException queryContextException = doPost( + final ErrorResponse errorResponse = doPost( new SqlQuery( "SELECT 1337", ResultFormat.OBJECT, @@ -1865,10 +1893,13 @@ public class SqlResourceTest extends CalciteTestBase null ) ).lhs; - Assert.assertNotNull(queryContextException); - Assert.assertEquals(QueryException.BAD_QUERY_CONTEXT_ERROR_CODE, queryContextException.getErrorCode()); - Assert.assertEquals(BadQueryContextException.ERROR_CLASS, queryContextException.getErrorClass()); - Assert.assertTrue(queryContextException.getMessage().contains("2000'")); + + validateLegacyQueryExceptionErrorResponse( + errorResponse, + QueryException.BAD_QUERY_CONTEXT_ERROR_CODE, + BadQueryContextException.ERROR_CLASS, + "2000'" + ); checkSqlRequestLog(false); Assert.assertTrue(lifecycleManager.getAll(sqlQueryId).isEmpty()); } @@ -1877,16 +1908,14 @@ public class SqlResourceTest extends CalciteTestBase public void testQueryContextKeyNotAllowed() throws Exception { Map queryContext = ImmutableMap.of(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY, "all"); - QueryException exception = postSyncForException( + ErrorResponse exception = postSyncForException( new SqlQuery("SELECT 1337", ResultFormat.OBJECT, false, false, false, queryContext, null), Status.BAD_REQUEST.getStatusCode() ); - Assert.assertNotNull(exception); - Assert.assertEquals(PlanningError.VALIDATION_ERROR.getErrorCode(), exception.getErrorCode()); - MatcherAssert.assertThat( - exception.getMessage(), - CoreMatchers.containsString("Cannot execute query with context parameter [sqlInsertSegmentGranularity]") + validateInvalidInputError( + exception, + "Query context parameter [sqlInsertSegmentGranularity] is not allowed" ); checkSqlRequestLog(false); } @@ -1915,7 +1944,7 @@ public class SqlResourceTest extends CalciteTestBase return new SqlQuery(sql, null, false, false, false, ImmutableMap.of(BaseQuery.SQL_QUERY_ID, sqlQueryId), null); } - private Pair>> doPost(final SqlQuery query) throws Exception + private Pair>> doPost(final SqlQuery query) throws Exception { return doPost(query, new TypeReference>>() { @@ -1923,7 +1952,7 @@ public class SqlResourceTest extends CalciteTestBase } // Returns either an error or a result, assuming the result is a JSON object. - private Pair doPost( + private Pair doPost( final SqlQuery query, final TypeReference typeReference ) throws Exception @@ -1931,38 +1960,30 @@ public class SqlResourceTest extends CalciteTestBase return doPost(query, req, typeReference); } - private Pair doPostRaw(final SqlQuery query) throws Exception + private Pair doPostRaw(final SqlQuery query) throws Exception { return doPostRaw(query, req); } - private Pair>> doPost(final SqlQuery query, MockHttpServletRequest req) - throws Exception - { - return doPost(query, req, new TypeReference>>() - { - }); - } - // Returns either an error or a result, assuming the result is a JSON object. @SuppressWarnings("unchecked") - private Pair doPost( + private Pair doPost( final SqlQuery query, final MockHttpServletRequest req, final TypeReference typeReference ) throws Exception { - final Pair pair = doPostRaw(query, req); + final Pair pair = doPostRaw(query, req); if (pair.rhs == null) { //noinspection unchecked - return (Pair) pair; + return (Pair) pair; } else { return Pair.of(pair.lhs, JSON_MAPPER.readValue(pair.rhs, typeReference)); } } // Returns either an error or a result. - private Pair doPostRaw(final SqlQuery query, final MockHttpServletRequest req) + private Pair doPostRaw(final SqlQuery query, final MockHttpServletRequest req) throws Exception { MockHttpServletResponse response = postForAsyncResponse(query, req); @@ -1970,7 +1991,7 @@ public class SqlResourceTest extends CalciteTestBase if (response.getStatus() == 200) { return Pair.of(null, new String(response.baos.toByteArray(), StandardCharsets.UTF_8)); } else { - return Pair.of(JSON_MAPPER.readValue(response.baos.toByteArray(), QueryException.class), null); + return Pair.of(JSON_MAPPER.readValue(response.baos.toByteArray(), ErrorResponse.class), null); } } @@ -2018,16 +2039,16 @@ public class SqlResourceTest extends CalciteTestBase return response; } - private QueryException postSyncForException(String s, int expectedStatus) throws IOException + private ErrorResponse postSyncForException(String s, int expectedStatus) throws IOException { return postSyncForException(createSimpleQueryWithId("id", s), expectedStatus); } - private QueryException postSyncForException(SqlQuery query, int expectedStatus) throws IOException + private ErrorResponse postSyncForException(SqlQuery query, int expectedStatus) throws IOException { final Response response = postForSyncResponse(query, req); assertStatusAndCommonHeaders(response, expectedStatus); - return deserializeResponse(response, QueryException.class); + return deserializeResponse(response, ErrorResponse.class); } private T deserializeResponse(Response resp, Class clazz) throws IOException @@ -2037,9 +2058,13 @@ public class SqlResourceTest extends CalciteTestBase private byte[] responseToByteArray(Response resp) throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ((StreamingOutput) resp.getEntity()).write(baos); - return baos.toByteArray(); + if (resp.getEntity() instanceof StreamingOutput) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ((StreamingOutput) resp.getEntity()).write(baos); + return baos.toByteArray(); + } else { + return JSON_MAPPER.writeValueAsBytes(resp.getEntity()); + } } private String getContentType(Response resp) @@ -2276,4 +2301,100 @@ public class SqlResourceTest extends CalciteTestBase }; } } + + private DruidException validateErrorResponse( + ErrorResponse errorResponse, + String errorCode, + DruidException.Persona targetPersona, + DruidException.Category category, + String messageContainsString + ) + { + Assert.assertNotNull(errorResponse); + + DruidException exception = errorResponse.getUnderlyingException(); + + Assert.assertEquals(errorCode, exception.getErrorCode()); + Assert.assertEquals(targetPersona, exception.getTargetPersona()); + Assert.assertEquals(category, exception.getCategory()); + if (messageContainsString == null) { + Assert.assertNull(exception.getMessage()); + } else { + MatcherAssert.assertThat(exception.getMessage(), CoreMatchers.containsString(messageContainsString)); + } + + return exception; + } + + private DruidException validateInvalidSqlError( + ErrorResponse response, + String containsString + ) + { + final DruidException exception = validateInvalidInputError(response, containsString); + Assert.assertEquals("sql", exception.getContextValue("sourceType")); + + return exception; + } + + @Nonnull + private DruidException validateInvalidInputError(ErrorResponse response, String containsString) + { + return validateErrorResponse( + response, + "invalidInput", + DruidException.Persona.USER, + DruidException.Category.INVALID_INPUT, + containsString + ); + } + + private DruidException validateLegacyQueryExceptionErrorResponse( + ErrorResponse errorResponse, + String legacyCode, + String errorClass, + String messageContainsString + ) + { + DruidException exception = validateErrorResponse( + errorResponse, + QueryExceptionCompat.ERROR_CODE, + DruidException.Persona.OPERATOR, + convertToCategory(legacyCode), + messageContainsString + ); + + Assert.assertEquals(legacyCode, exception.getContextValue("legacyErrorCode")); + Assert.assertEquals(errorClass, exception.getContextValue("errorClass")); + + return exception; + } + + private static DruidException.Category convertToCategory(String legacyErrorCode) + { + // This code is copied from QueryExceptionCompat at the time of writing. This is because these mappings + // are fundamentally part of the API, so reusing the code from there runs the risk that changes in the mapping + // would change the API but not break the unit tests. So, the unit test uses its own mapping to ensure + // that we are validating and aware of API-affecting changes. + switch (QueryException.fromErrorCode(legacyErrorCode)) { + case USER_ERROR: + return DruidException.Category.INVALID_INPUT; + case UNAUTHORIZED: + return DruidException.Category.UNAUTHORIZED; + case CAPACITY_EXCEEDED: + return DruidException.Category.CAPACITY_EXCEEDED; + case QUERY_RUNTIME_FAILURE: + return DruidException.Category.RUNTIME_FAILURE; + case CANCELED: + return DruidException.Category.CANCELED; + case UNKNOWN: + return DruidException.Category.UNCATEGORIZED; + case UNSUPPORTED: + return DruidException.Category.UNSUPPORTED; + case TIMEOUT: + return DruidException.Category.TIMEOUT; + default: + return DruidException.Category.UNCATEGORIZED; + } + } } From 50461c3bd58da60ec3cdfbc9a962d4a594ec7c6a Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 19 Jun 2023 14:27:35 +0530 Subject: [PATCH 07/74] Enable smartSegmentLoading on the Coordinator (#13197) This commit does a complete revamp of the coordinator to address problem areas: - Stability: Fix several bugs, add capabilities to prioritize and cancel load queue items - Visibility: Add new metrics, improve logs, revamp `CoordinatorRunStats` - Configuration: Add dynamic config `smartSegmentLoading` to automatically set optimal values for all segment loading configs such as `maxSegmentsToMove`, `replicationThrottleLimit` and `maxSegmentsInNodeLoadingQueue`. Changed classes: - Add `StrategicSegmentAssigner` to make assignment decisions for load, replicate and move - Add `SegmentAction` to distinguish between load, replicate, drop and move operations - Add `SegmentReplicationStatus` to capture current state of replication of all used segments - Add `SegmentLoadingConfig` to contain recomputed dynamic config values - Simplify classes `LoadRule`, `BroadcastRule` - Simplify the `BalancerStrategy` and `CostBalancerStrategy` - Add several new methods to `ServerHolder` to track loaded and queued segments - Refactor `DruidCoordinator` Impact: - Enable `smartSegmentLoading` by default. With this enabled, none of the following dynamic configs need to be set: `maxSegmentsToMove`, `replicationThrottleLimit`, `maxSegmentsInNodeLoadingQueue`, `useRoundRobinSegmentAssignment`, `emitBalancingStats` and `replicantLifetime`. - Coordinator reports richer metrics and produces cleaner and more informative logs - Coordinator uses an unlimited load queue for all serves, and makes better assignment decisions --- .../BalancerStrategyBenchmark.java | 41 +- .../CachingCostBalancerStrategyBenchmark.java | 3 +- .../CostBalancerStrategyBenchmark.java | 1 + .../utils/DruidClusterAdminClient.java | 4 +- .../utils/DruidClusterAdminClient.java | 4 +- .../org/apache/druid/query/DruidMetrics.java | 6 - .../metadata/SqlSegmentsMetadataManager.java | 3 +- .../server/coordinator/BalancerStrategy.java | 186 --- .../coordinator/CoordinatorDynamicConfig.java | 386 +++-- .../server/coordinator/CoordinatorStats.java | 226 --- .../server/coordinator/DruidCluster.java | 155 +- .../server/coordinator/DruidCoordinator.java | 767 ++++----- .../DruidCoordinatorRuntimeParams.java | 268 ++- .../coordinator/ReplicationThrottler.java | 182 -- .../coordinator/SegmentReplicantLookup.java | 174 -- .../server/coordinator/ServerHolder.java | 390 +++-- .../{ => balancer}/BalancerSegmentHolder.java | 30 +- .../balancer/BalancerStrategy.java | 89 + .../BalancerStrategyFactory.java | 10 +- .../CachingCostBalancerStrategy.java | 21 +- .../CachingCostBalancerStrategyConfig.java | 2 +- .../CachingCostBalancerStrategyFactory.java | 3 +- .../{cost => balancer}/ClusterCostCache.java | 2 +- .../{ => balancer}/CostBalancerStrategy.java | 179 +- .../CostBalancerStrategyFactory.java | 2 +- .../DiskNormalizedCostBalancerStrategy.java | 3 +- ...NormalizedCostBalancerStrategyFactory.java | 2 +- .../RandomBalancerStrategy.java | 47 +- .../RandomBalancerStrategyFactory.java | 2 +- .../ReservoirSegmentSampler.java | 54 +- .../{cost => balancer}/SegmentsCostCache.java | 3 +- .../{cost => balancer}/ServerCostCache.java | 2 +- .../balancer/TierSegmentBalancer.java | 237 +++ .../{cost => balancer}/package-info.java | 2 +- .../coordinator/duty/BalanceSegments.java | 288 +--- .../duty/CollectSegmentAndServerStats.java | 154 ++ .../coordinator/duty/CompactSegments.java | 641 ++++--- .../duty/EmitClusterStatsAndMetrics.java | 525 ------ .../coordinator/duty/LogUsedSegments.java | 53 - ... => MarkOvershadowedSegmentsAsUnused.java} | 64 +- .../server/coordinator/duty/RunRules.java | 136 +- .../duty/SegmentCompactionUtil.java | 49 - .../duty/UnloadUnusedSegments.java | 113 +- .../{ => loading}/CuratorLoadQueuePeon.java | 167 +- .../{ => loading}/HttpLoadQueuePeon.java | 335 ++-- .../{ => loading}/LoadPeonCallback.java | 2 +- .../{ => loading}/LoadQueuePeon.java | 42 +- .../{ => loading}/LoadQueueTaskMaster.java | 3 +- .../loading/ReplicationThrottler.java | 94 ++ .../RoundRobinServerSelector.java | 4 +- .../coordinator/loading/SegmentAction.java | 81 + .../coordinator/loading/SegmentHolder.java | 178 ++ .../loading/SegmentLoadQueueManager.java | 159 ++ .../loading/SegmentLoadingConfig.java | 153 ++ .../loading/SegmentReplicaCount.java | 167 ++ .../loading/SegmentReplicaCountMap.java | 108 ++ .../loading/SegmentReplicationStatus.java | 84 + .../loading/SegmentStatusInTier.java | 78 + .../loading/StrategicSegmentAssigner.java | 606 +++++++ .../rules/BroadcastDistributionRule.java | 141 +- .../server/coordinator/rules/DropRule.java | 15 +- .../coordinator/rules/ForeverLoadRule.java | 4 +- .../server/coordinator/rules/LoadRule.java | 535 +----- .../druid/server/coordinator/rules/Rule.java | 61 +- ...RuleMap.java => SegmentActionHandler.java} | 42 +- .../stats/CoordinatorRunStats.java | 273 +++ .../coordinator/stats/CoordinatorStat.java | 99 ++ .../server/coordinator/stats/Dimension.java} | 34 +- .../server/coordinator/stats/RowKey.java | 103 ++ .../druid/server/coordinator/stats/Stats.java | 141 ++ .../CoordinatorDynamicConfigsResource.java | 6 +- .../server/http/CoordinatorResource.java | 64 +- .../server/http/DataSourcesResource.java | 4 +- .../druid/server/http/MetadataResource.java | 25 +- .../server/http/SegmentListerResource.java | 2 +- .../appenderator/TestUsedSegmentChecker.java | 4 +- .../coordinator/BalanceSegmentsProfiler.java | 59 +- .../coordinator/BalanceSegmentsTest.java | 814 --------- .../coordinator/BalanceSegmentsTester.java | 76 - .../coordinator/CoordinatorRunStatsTest.java | 211 +++ .../coordinator/CoordinatorStatsTest.java | 233 --- .../CostBalancerStrategyBenchmark.java | 5 +- .../coordinator/CreateDataSegments.java | 15 +- .../CuratorDruidCoordinatorTest.java | 206 +-- .../coordinator/DruidClusterBuilder.java | 68 - .../server/coordinator/DruidClusterTest.java | 113 +- .../coordinator/DruidCoordinatorTest.java | 300 +--- .../coordinator/HttpLoadQueuePeonTest.java | 203 --- .../RoundRobinServerSelectorTest.java | 47 +- .../server/coordinator/RunRulesTest.java | 1480 ----------------- .../server/coordinator/ServerHolderTest.java | 9 +- .../{ => balancer}/BalancerStrategyTest.java | 28 +- .../CachingCostBalancerStrategyTest.java | 13 +- .../CostBalancerStrategyTest.java | 26 +- ...iskNormalizedCostBalancerStrategyTest.java | 8 +- .../ReservoirSegmentSamplerTest.java | 63 +- .../SegmentsCostCacheTest.java | 2 +- .../coordinator/duty/BalanceSegmentsTest.java | 491 ++++++ .../CollectSegmentAndServerStatsTest.java | 71 + .../coordinator/duty/CompactSegmentsTest.java | 220 ++- .../duty/EmitClusterStatsAndMetricsTest.java | 124 -- .../MarkAsUnusedOvershadowedSegmentsTest.java | 141 -- .../MarkOvershadowedSegmentsAsUnusedTest.java | 104 ++ .../duty/NewestSegmentFirstPolicyTest.java | 19 +- .../server/coordinator/duty/RunRulesTest.java | 1256 ++++++++++++++ .../duty/UnloadUnusedSegmentsTest.java | 106 +- .../loading/HttpLoadQueuePeonTest.java | 382 +++++ .../{ => loading}/LoadQueuePeonTest.java | 14 +- .../{ => loading}/LoadQueuePeonTester.java | 17 +- .../rules/BroadcastDistributionRuleTest.java | 155 +- .../coordinator/rules/LoadRuleTest.java | 852 ++++------ .../simulate/BalancingStrategiesTest.java | 18 +- .../CoordinatorSimulationBaseTest.java | 64 +- .../CoordinatorSimulationBuilder.java | 28 +- .../simulate/RoundRobinAssignmentTest.java | 3 +- .../simulate/SegmentBalancingTest.java | 106 +- .../simulate/SegmentLoadingNegativeTest.java | 220 --- .../simulate/SegmentLoadingTest.java | 471 +++++- .../simulate/TestSegmentsMetadataManager.java | 14 +- .../http/CoordinatorDynamicConfigTest.java | 149 +- .../server/http/DataSourcesResourceTest.java | 4 +- .../server/http/MetadataResourceTest.java | 145 +- .../org/apache/druid/cli/CliCoordinator.java | 6 +- 123 files changed, 8898 insertions(+), 9584 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/CoordinatorStats.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/BalancerSegmentHolder.java (65%) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/BalancerStrategyFactory.java (74%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/CachingCostBalancerStrategy.java (82%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/CachingCostBalancerStrategyConfig.java (95%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/CachingCostBalancerStrategyFactory.java (98%) rename server/src/main/java/org/apache/druid/server/coordinator/{cost => balancer}/ClusterCostCache.java (98%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/CostBalancerStrategy.java (65%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/CostBalancerStrategyFactory.java (95%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/DiskNormalizedCostBalancerStrategy.java (94%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/DiskNormalizedCostBalancerStrategyFactory.java (95%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/RandomBalancerStrategy.java (50%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/RandomBalancerStrategyFactory.java (95%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => balancer}/ReservoirSegmentSampler.java (74%) rename server/src/main/java/org/apache/druid/server/coordinator/{cost => balancer}/SegmentsCostCache.java (99%) rename server/src/main/java/org/apache/druid/server/coordinator/{cost => balancer}/ServerCostCache.java (98%) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java rename server/src/main/java/org/apache/druid/server/coordinator/{cost => balancer}/package-info.java (94%) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/LogUsedSegments.java rename server/src/main/java/org/apache/druid/server/coordinator/duty/{MarkAsUnusedOvershadowedSegments.java => MarkOvershadowedSegmentsAsUnused.java} (63%) delete mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/SegmentCompactionUtil.java rename server/src/main/java/org/apache/druid/server/coordinator/{ => loading}/CuratorLoadQueuePeon.java (77%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => loading}/HttpLoadQueuePeon.java (65%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => loading}/LoadPeonCallback.java (96%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => loading}/LoadQueuePeon.java (52%) rename server/src/main/java/org/apache/druid/server/coordinator/{ => loading}/LoadQueueTaskMaster.java (95%) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/ReplicationThrottler.java rename server/src/main/java/org/apache/druid/server/coordinator/{ => loading}/RoundRobinServerSelector.java (96%) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentAction.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicaCount.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicaCountMap.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicationStatus.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentStatusInTier.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java rename server/src/main/java/org/apache/druid/server/coordinator/rules/{RuleMap.java => SegmentActionHandler.java} (53%) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorStat.java rename server/src/{test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java => main/java/org/apache/druid/server/coordinator/stats/Dimension.java} (58%) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTester.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordinator/CoordinatorStatsTest.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordinator/RunRulesTest.java rename server/src/test/java/org/apache/druid/server/coordinator/{ => balancer}/BalancerStrategyTest.java (80%) rename server/src/test/java/org/apache/druid/server/coordinator/{cost => balancer}/CachingCostBalancerStrategyTest.java (92%) rename server/src/test/java/org/apache/druid/server/coordinator/{ => balancer}/CostBalancerStrategyTest.java (94%) rename server/src/test/java/org/apache/druid/server/coordinator/{ => balancer}/DiskNormalizedCostBalancerStrategyTest.java (94%) rename server/src/test/java/org/apache/druid/server/coordinator/{ => balancer}/ReservoirSegmentSamplerTest.java (75%) rename server/src/test/java/org/apache/druid/server/coordinator/{cost => balancer}/SegmentsCostCacheTest.java (99%) create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetricsTest.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegmentsTest.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java rename server/src/test/java/org/apache/druid/server/coordinator/{ => loading}/LoadQueuePeonTest.java (96%) rename server/src/test/java/org/apache/druid/server/coordinator/{ => loading}/LoadQueuePeonTester.java (85%) delete mode 100644 server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingNegativeTest.java diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyBenchmark.java index af6007c8ac8..0bf5611998d 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyBenchmark.java @@ -20,13 +20,14 @@ package org.apache.druid.server.coordinator; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.balancer.BalancerSegmentHolder; +import org.apache.druid.server.coordinator.balancer.ReservoirSegmentSampler; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.openjdk.jmh.annotations.Benchmark; @@ -63,7 +64,7 @@ public class BalancerStrategyBenchmark private static final Interval TEST_SEGMENT_INTERVAL = Intervals.of("2012-03-15T00:00:00.000/2012-03-16T00:00:00.000"); private static final int NUMBER_OF_SERVERS = 20; - @Param({"default", "50percentOfSegmentsToConsiderPerMove", "useBatchedSegmentSampler"}) + @Param({"default", "useBatchedSegmentSampler"}) private String mode; @Param({"10000", "100000", "1000000"}) @@ -73,26 +74,13 @@ public class BalancerStrategyBenchmark private int maxSegmentsToMove; private final List serverHolders = new ArrayList<>(); - private boolean useBatchedSegmentSampler; private int reservoirSize = 1; - private double percentOfSegmentsToConsider = 100; - private final BalancerStrategy balancerStrategy = new CostBalancerStrategy( - MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalancerStrategyBenchmark-%d")) - ); @Setup(Level.Trial) public void setup() { - switch (mode) { - case "50percentOfSegmentsToConsiderPerMove": - percentOfSegmentsToConsider = 50; - useBatchedSegmentSampler = false; - break; - case "useBatchedSegmentSampler": - reservoirSize = maxSegmentsToMove; - useBatchedSegmentSampler = true; - break; - default: + if ("useBatchedSegmentSampler".equals(mode)) { + reservoirSize = maxSegmentsToMove; } List> segmentList = new ArrayList<>(NUMBER_OF_SERVERS); @@ -131,20 +119,9 @@ public class BalancerStrategyBenchmark @Benchmark public void pickSegmentsToMove(Blackhole blackhole) { - Iterator iterator; - if (useBatchedSegmentSampler) { - iterator = balancerStrategy.pickSegmentsToMove( - serverHolders, - Collections.emptySet(), - reservoirSize - ); - } else { - iterator = balancerStrategy.pickSegmentsToMove( - serverHolders, - Collections.emptySet(), - percentOfSegmentsToConsider - ); - } + Iterator iterator = ReservoirSegmentSampler + .pickMovableSegmentsFrom(serverHolders, reservoirSize, ServerHolder::getServedSegments, Collections.emptySet()) + .iterator(); for (int i = 0; i < maxSegmentsToMove && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyBenchmark.java index 73f8ba8a6a9..29d164331ed 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyBenchmark.java @@ -21,7 +21,8 @@ package org.apache.druid.server.coordinator; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.coordinator.cost.SegmentsCostCache; +import org.apache.druid.server.coordinator.balancer.CostBalancerStrategy; +import org.apache.druid.server.coordinator.balancer.SegmentsCostCache; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyBenchmark.java index 8b40d30f782..e1a05aea079 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyBenchmark.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.server.coordinator.balancer.CostBalancerStrategy; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java index 60bd64ef847..9c15eef30c5 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java @@ -124,7 +124,7 @@ public class DruidClusterAdminClient { waitUntilInstanceReady(config.getCoordinatorUrl()); postDynamicConfig(CoordinatorDynamicConfig.builder() - .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1) + .withMarkSegmentAsUnusedDelayMillis(1) .build()); } @@ -132,7 +132,7 @@ public class DruidClusterAdminClient { waitUntilInstanceReady(config.getCoordinatorTwoUrl()); postDynamicConfig(CoordinatorDynamicConfig.builder() - .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1) + .withMarkSegmentAsUnusedDelayMillis(1) .build()); } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java index 01e9bed4995..6b98306ca0e 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/DruidClusterAdminClient.java @@ -123,7 +123,7 @@ public class DruidClusterAdminClient { waitUntilInstanceReady(config.getCoordinatorUrl()); postDynamicConfig(CoordinatorDynamicConfig.builder() - .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1) + .withMarkSegmentAsUnusedDelayMillis(1) .build()); } @@ -131,7 +131,7 @@ public class DruidClusterAdminClient { waitUntilInstanceReady(config.getCoordinatorTwoUrl()); postDynamicConfig(CoordinatorDynamicConfig.builder() - .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1) + .withMarkSegmentAsUnusedDelayMillis(1) .build()); } diff --git a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java index 5caf90d3fda..3dc8685fc00 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java @@ -43,12 +43,6 @@ public class DruidMetrics public static final String TASK_TYPE = "taskType"; public static final String TASK_STATUS = "taskStatus"; - public static final String SERVER = "server"; - public static final String TIER = "tier"; - - public static final String DUTY = "duty"; - public static final String DUTY_GROUP = "dutyGroup"; - public static final String STREAM = "stream"; public static final String PARTITION = "partition"; diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java index 7ef396174ec..d98c5d4d086 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java @@ -558,8 +558,7 @@ public class SqlSegmentsMetadataManager implements SegmentsMetadataManager private int doMarkAsUsedNonOvershadowedSegments(String dataSourceName, @Nullable Interval interval) { final List unusedSegments = new ArrayList<>(); - final SegmentTimeline timeline = - SegmentTimeline.forSegments(Collections.emptyIterator()); + final SegmentTimeline timeline = new SegmentTimeline(); connector.inReadOnlyTransaction( (handle, status) -> { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java deleted file mode 100644 index 4a48137a888..00000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java +++ /dev/null @@ -1,186 +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.druid.server.coordinator; - -import org.apache.druid.server.coordinator.duty.BalanceSegments; -import org.apache.druid.timeline.DataSegment; - -import javax.annotation.Nullable; -import java.util.Iterator; -import java.util.List; -import java.util.NavigableSet; -import java.util.Set; - -/** - * This interface describes the coordinator balancing strategy, which is responsible for making decisions on where - * to place {@link DataSegment}s on historical servers (described by {@link ServerHolder}). The balancing strategy - * is used by {@link org.apache.druid.server.coordinator.rules.LoadRule} to assign and drop segments, and by - * {@link BalanceSegments} to migrate segments between historicals. - */ -public interface BalancerStrategy -{ - /** - * Find the best server to move a {@link DataSegment} to according the balancing strategy. - * @param proposalSegment segment to move - * @param serverHolders servers to consider as move destinations - * @return The server to move to, or null if no move should be made or no server is suitable - */ - @Nullable - ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List serverHolders); - - /** - * Find the best server on which to place a {@link DataSegment} replica according to the balancing strategy - * @param proposalSegment segment to replicate - * @param serverHolders servers to consider as replica holders - * @return The server to replicate to, or null if no suitable server is found - */ - @Nullable - ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List serverHolders); - - /** - * Pick the best segments to move from one of the supplied set of servers according to the balancing strategy. - * - * @param serverHolders set of historicals to consider for moving segments - * @param broadcastDatasources Datasources that contain segments which were loaded via broadcast rules. - * Balancing strategies should avoid rebalancing segments for such datasources, since - * they should be loaded on all servers anyway. - * NOTE: this should really be handled on a per-segment basis, to properly support - * the interval or period-based broadcast rules. For simplicity of the initial - * implementation, only forever broadcast rules are supported. - * @param reservoirSize the reservoir size maintained by the Reservoir Sampling algorithm. - * @return Iterator for set of {@link BalancerSegmentHolder} containing segment to move and server they currently - * reside on, or empty if there are no segments to pick from (i. e. all provided serverHolders are empty). - */ - default Iterator pickSegmentsToMove( - List serverHolders, - Set broadcastDatasources, - int reservoirSize - ) - { - return new Iterator() - { - private Iterator it = sample(); - private Iterator sample() - { - return ReservoirSegmentSampler.getRandomBalancerSegmentHolders( - serverHolders, - broadcastDatasources, - reservoirSize - ).iterator(); - } - - @Override - public boolean hasNext() - { - if (it.hasNext()) { - return true; - } - it = sample(); - return it.hasNext(); - } - - @Override - public BalancerSegmentHolder next() - { - return it.next(); - } - }; - } - - /** - * Pick the best segments to move from one of the supplied set of servers according to the balancing strategy. - * - * @param serverHolders set of historicals to consider for moving segments - * @param broadcastDatasources Datasources that contain segments which were loaded via broadcast rules. - * Balancing strategies should avoid rebalancing segments for such datasources, since - * they should be loaded on all servers anyway. - * NOTE: this should really be handled on a per-segment basis, to properly support - * the interval or period-based broadcast rules. For simplicity of the initial - * implementation, only forever broadcast rules are supported. - * @param percentOfSegmentsToConsider The percentage of the total number of segments that we will consider when - * choosing which segment to move. {@link CoordinatorDynamicConfig} defines a - * config percentOfSegmentsToConsiderPerMove that will be used as an argument - * for implementations of this method. - * @return Iterator for set of {@link BalancerSegmentHolder} containing segment to move and server they currently - * reside on, or empty if there are no segments to pick from (i. e. all provided serverHolders are empty). - * - * @deprecated Use {@link #pickSegmentsToMove(List, Set, int)} instead as it is - * a much more performant sampling method which does not allow duplicates. This - * method will be removed in future releases. - */ - @Deprecated - default Iterator pickSegmentsToMove( - List serverHolders, - Set broadcastDatasources, - double percentOfSegmentsToConsider - ) - { - return new Iterator() - { - private BalancerSegmentHolder next = sample(); - - private BalancerSegmentHolder sample() - { - return ReservoirSegmentSampler.getRandomBalancerSegmentHolder( - serverHolders, - broadcastDatasources, - percentOfSegmentsToConsider - ); - } - - @Override - public boolean hasNext() - { - return next != null; - } - - @Override - public BalancerSegmentHolder next() - { - BalancerSegmentHolder ret = next; - next = sample(); - return ret; - } - }; - } - - /** - * Returns an iterator for a set of servers to drop from, ordered by preference of which server to drop from first - * for a given drop strategy. One or more segments may be dropped, depending on how much the segment is - * over-replicated. - * @param toDropSegment segment to drop from one or more servers - * @param serverHolders set of historicals to consider dropping from - * @return Iterator for set of historicals, ordered by drop preference - */ - default Iterator pickServersToDrop(DataSegment toDropSegment, NavigableSet serverHolders) - { - // By default, use the reverse order to get the holders with least available size first. - return serverHolders.descendingIterator(); - } - - /** - * Add balancing strategy stats during the 'balanceTier' operation of - * {@link BalanceSegments} to be included - * @param tier historical tier being balanced - * @param stats stats object to add balancing strategy stats to - * @param serverHolderList servers in tier being balanced - */ - void emitStats(String tier, CoordinatorStats stats, List serverHolderList); -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index b1b49da3814..cd0e2b67e74 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -27,13 +27,18 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.duty.KillUnusedSegments; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.server.coordinator.stats.Dimension; import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.validation.constraints.Max; import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; import java.util.Collection; +import java.util.EnumMap; import java.util.HashSet; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -50,7 +55,7 @@ public class CoordinatorDynamicConfig { public static final String CONFIG_KEY = "coordinator.config"; - private final long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; + private final long markSegmentAsUnusedDelayMillis; private final long mergeBytesLimit; private final int mergeSegmentsLimit; private final int maxSegmentsToMove; @@ -63,12 +68,18 @@ public class CoordinatorDynamicConfig private final int balancerComputeThreads; private final boolean emitBalancingStats; private final boolean useRoundRobinSegmentAssignment; + private final boolean smartSegmentLoading; /** * List of specific data sources for which kill tasks are sent in {@link KillUnusedSegments}. */ private final Set specificDataSourcesToKillUnusedSegmentsIn; private final Set decommissioningNodes; + + private final Map debugDimensions; + private final Map validDebugDimensions; + + @Deprecated private final int decommissioningMaxPercentOfMaxSegmentsToMove; /** @@ -102,6 +113,7 @@ public class CoordinatorDynamicConfig * be set to put a hard upper limit on the number of replicants loaded. It is a tool that can help prevent * long delays in new data loads after events such as a Historical server leaving the cluster. */ + @Deprecated private final int maxNonPrimaryReplicantsToLoad; private static final Logger log = new Logger(CoordinatorDynamicConfig.class); @@ -111,7 +123,7 @@ public class CoordinatorDynamicConfig // Keeping the legacy 'millisToWaitBeforeDeleting' property name for backward compatibility. When the project is // updated to Jackson 2.9 it could be changed, see https://github.com/apache/druid/issues/7152 @JsonProperty("millisToWaitBeforeDeleting") - long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, + long markSegmentAsUnusedDelayMillis, @JsonProperty("mergeBytesLimit") long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") int mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") int maxSegmentsToMove, @@ -137,14 +149,17 @@ public class CoordinatorDynamicConfig @JsonProperty("pauseCoordination") boolean pauseCoordination, @JsonProperty("replicateAfterLoadTimeout") boolean replicateAfterLoadTimeout, @JsonProperty("maxNonPrimaryReplicantsToLoad") @Nullable Integer maxNonPrimaryReplicantsToLoad, - @JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment + @JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment, + @JsonProperty("smartSegmentLoading") @Nullable Boolean smartSegmentLoading, + @JsonProperty("debugDimensions") @Nullable Map debugDimensions ) { - this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = - leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; + this.markSegmentAsUnusedDelayMillis = + markSegmentAsUnusedDelayMillis; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; + this.smartSegmentLoading = Builder.valueOrDefault(smartSegmentLoading, Defaults.SMART_SEGMENT_LOADING); if (percentOfSegmentsToConsiderPerMove == null) { log.debug( @@ -152,36 +167,37 @@ public class CoordinatorDynamicConfig + "reflect this configuration being added to Druid in a recent release. Druid is defaulting the value " + "to the Druid default of %f. It is recommended that you re-submit your dynamic config with your " + "desired value for percentOfSegmentsToConsideredPerMove", - Builder.DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE + Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE ); - percentOfSegmentsToConsiderPerMove = Builder.DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE; + percentOfSegmentsToConsiderPerMove = Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE; } Preconditions.checkArgument( percentOfSegmentsToConsiderPerMove > 0 && percentOfSegmentsToConsiderPerMove <= 100, - "percentOfSegmentsToConsiderPerMove should be between 1 and 100!" + "'percentOfSegmentsToConsiderPerMove' should be between 1 and 100" ); this.percentOfSegmentsToConsiderPerMove = percentOfSegmentsToConsiderPerMove; - if (useBatchedSegmentSampler == null) { - this.useBatchedSegmentSampler = Builder.DEFAULT_USE_BATCHED_SEGMENT_SAMPLER; - } else { - this.useBatchedSegmentSampler = useBatchedSegmentSampler; - } + this.useBatchedSegmentSampler = Builder.valueOrDefault( + useBatchedSegmentSampler, + Defaults.USE_BATCHED_SEGMENT_SAMPLER + ); this.replicantLifetime = replicantLifetime; this.replicationThrottleLimit = replicationThrottleLimit; this.balancerComputeThreads = Math.max(balancerComputeThreads, 1); this.emitBalancingStats = emitBalancingStats; - this.specificDataSourcesToKillUnusedSegmentsIn = parseJsonStringOrArray(specificDataSourcesToKillUnusedSegmentsIn); - this.dataSourcesToNotKillStalePendingSegmentsIn = - parseJsonStringOrArray(dataSourcesToNotKillStalePendingSegmentsIn); - this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue == null - ? Builder.DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE - : maxSegmentsInNodeLoadingQueue; + this.specificDataSourcesToKillUnusedSegmentsIn + = parseJsonStringOrArray(specificDataSourcesToKillUnusedSegmentsIn); + this.dataSourcesToNotKillStalePendingSegmentsIn + = parseJsonStringOrArray(dataSourcesToNotKillStalePendingSegmentsIn); + this.maxSegmentsInNodeLoadingQueue = Builder.valueOrDefault( + maxSegmentsInNodeLoadingQueue, + Defaults.MAX_SEGMENTS_IN_NODE_LOADING_QUEUE + ); this.decommissioningNodes = parseJsonStringOrArray(decommissioningNodes); Preconditions.checkArgument( decommissioningMaxPercentOfMaxSegmentsToMove >= 0 && decommissioningMaxPercentOfMaxSegmentsToMove <= 100, - "decommissioningMaxPercentOfMaxSegmentsToMove should be in range [0, 100]" + "'decommissioningMaxPercentOfMaxSegmentsToMove' should be in range [0, 100]" ); this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove; @@ -194,9 +210,9 @@ public class CoordinatorDynamicConfig + "reflect this configuration being added to Druid in a recent release. Druid is defaulting the value " + "to the Druid default of %d. It is recommended that you re-submit your dynamic config with your " + "desired value for maxNonPrimaryReplicantsToLoad", - Builder.DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD + Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD ); - maxNonPrimaryReplicantsToLoad = Builder.DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD; + maxNonPrimaryReplicantsToLoad = Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD; } Preconditions.checkArgument( maxNonPrimaryReplicantsToLoad >= 0, @@ -204,11 +220,29 @@ public class CoordinatorDynamicConfig ); this.maxNonPrimaryReplicantsToLoad = maxNonPrimaryReplicantsToLoad; - if (useRoundRobinSegmentAssignment == null) { - this.useRoundRobinSegmentAssignment = Builder.DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT; - } else { - this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment; + this.useRoundRobinSegmentAssignment = Builder.valueOrDefault( + useRoundRobinSegmentAssignment, + Defaults.USE_ROUND_ROBIN_ASSIGNMENT + ); + this.debugDimensions = debugDimensions; + this.validDebugDimensions = validateDebugDimensions(debugDimensions); + } + + private Map validateDebugDimensions(Map debugDimensions) + { + final Map validDebugDimensions = new EnumMap<>(Dimension.class); + if (debugDimensions == null || debugDimensions.isEmpty()) { + return validDebugDimensions; } + + for (Dimension dimension : Dimension.values()) { + final String dimensionValue = debugDimensions.get(dimension.reportedName()); + if (dimensionValue != null) { + validDebugDimensions.put(dimension, dimensionValue); + } + } + + return validDebugDimensions; } private static Set parseJsonStringOrArray(Object jsonStringOrArray) @@ -224,7 +258,7 @@ public class CoordinatorDynamicConfig } return result; } else if (jsonStringOrArray instanceof Collection) { - return ImmutableSet.copyOf(((Collection) jsonStringOrArray)); + return ImmutableSet.copyOf((Collection) jsonStringOrArray); } else { return ImmutableSet.of(); } @@ -246,9 +280,9 @@ public class CoordinatorDynamicConfig } @JsonProperty("millisToWaitBeforeDeleting") - public long getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() + public long getMarkSegmentAsUnusedDelayMillis() { - return leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; + return markSegmentAsUnusedDelayMillis; } @JsonProperty @@ -337,6 +371,12 @@ public class CoordinatorDynamicConfig return useRoundRobinSegmentAssignment; } + @JsonProperty + public boolean isSmartSegmentLoading() + { + return smartSegmentLoading; + } + /** * List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning' * servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate @@ -348,7 +388,18 @@ public class CoordinatorDynamicConfig public Set getDecommissioningNodes() { return decommissioningNodes; + } + @JsonProperty + public Map getDebugDimensions() + { + return debugDimensions; + } + + @JsonIgnore + public Map getValidatedDebugDimensions() + { + return validDebugDimensions; } /** @@ -365,6 +416,7 @@ public class CoordinatorDynamicConfig */ @Min(0) @Max(100) + @Deprecated @JsonProperty public int getDecommissioningMaxPercentOfMaxSegmentsToMove() { @@ -384,6 +436,7 @@ public class CoordinatorDynamicConfig } @Min(0) + @Deprecated @JsonProperty public int getMaxNonPrimaryReplicantsToLoad() { @@ -395,7 +448,7 @@ public class CoordinatorDynamicConfig { return "CoordinatorDynamicConfig{" + "leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments=" - + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments + + + markSegmentAsUnusedDelayMillis + ", mergeBytesLimit=" + mergeBytesLimit + ", mergeSegmentsLimit=" + mergeSegmentsLimit + ", maxSegmentsToMove=" + maxSegmentsToMove + @@ -428,66 +481,37 @@ public class CoordinatorDynamicConfig CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o; - if (leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments != - that.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments) { - return false; - } - if (mergeBytesLimit != that.mergeBytesLimit) { - return false; - } - if (mergeSegmentsLimit != that.mergeSegmentsLimit) { - return false; - } - if (maxSegmentsToMove != that.maxSegmentsToMove) { - return false; - } - if (percentOfSegmentsToConsiderPerMove != that.percentOfSegmentsToConsiderPerMove) { - return false; - } - if (useBatchedSegmentSampler != that.useBatchedSegmentSampler) { - return false; - } - if (replicantLifetime != that.replicantLifetime) { - return false; - } - if (replicationThrottleLimit != that.replicationThrottleLimit) { - return false; - } - if (balancerComputeThreads != that.balancerComputeThreads) { - return false; - } - if (emitBalancingStats != that.emitBalancingStats) { - return false; - } - if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) { - return false; - } - if (!Objects.equals(specificDataSourcesToKillUnusedSegmentsIn, that.specificDataSourcesToKillUnusedSegmentsIn)) { - return false; - } - if (!Objects.equals(dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn)) { - return false; - } - if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) { - return false; - } - if (pauseCoordination != that.pauseCoordination) { - return false; - } - if (replicateAfterLoadTimeout != that.replicateAfterLoadTimeout) { - return false; - } - if (maxNonPrimaryReplicantsToLoad != that.maxNonPrimaryReplicantsToLoad) { - return false; - } - return decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove; + return markSegmentAsUnusedDelayMillis == that.markSegmentAsUnusedDelayMillis + && mergeBytesLimit == that.mergeBytesLimit + && mergeSegmentsLimit == that.mergeSegmentsLimit + && maxSegmentsToMove == that.maxSegmentsToMove + && percentOfSegmentsToConsiderPerMove == that.percentOfSegmentsToConsiderPerMove + && decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove + && useBatchedSegmentSampler == that.useBatchedSegmentSampler + && balancerComputeThreads == that.balancerComputeThreads + && emitBalancingStats == that.emitBalancingStats + && replicantLifetime == that.replicantLifetime + && replicationThrottleLimit == that.replicationThrottleLimit + && replicateAfterLoadTimeout == that.replicateAfterLoadTimeout + && maxSegmentsInNodeLoadingQueue == that.maxSegmentsInNodeLoadingQueue + && maxNonPrimaryReplicantsToLoad == that.maxNonPrimaryReplicantsToLoad + && useRoundRobinSegmentAssignment == that.useRoundRobinSegmentAssignment + && pauseCoordination == that.pauseCoordination + && Objects.equals( + specificDataSourcesToKillUnusedSegmentsIn, + that.specificDataSourcesToKillUnusedSegmentsIn) + && Objects.equals( + dataSourcesToNotKillStalePendingSegmentsIn, + that.dataSourcesToNotKillStalePendingSegmentsIn) + && Objects.equals(decommissioningNodes, that.decommissioningNodes) + && Objects.equals(debugDimensions, that.debugDimensions); } @Override public int hashCode() { return Objects.hash( - leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, + markSegmentAsUnusedDelayMillis, mergeBytesLimit, mergeSegmentsLimit, maxSegmentsToMove, @@ -503,7 +527,8 @@ public class CoordinatorDynamicConfig decommissioningNodes, decommissioningMaxPercentOfMaxSegmentsToMove, pauseCoordination, - maxNonPrimaryReplicantsToLoad + maxNonPrimaryReplicantsToLoad, + debugDimensions ); } @@ -512,27 +537,30 @@ public class CoordinatorDynamicConfig return new Builder(); } + private static class Defaults + { + static final long LEADING_MILLIS_BEFORE_MARK_UNUSED = TimeUnit.MINUTES.toMillis(15); + static final long MERGE_BYTES_LIMIT = 524_288_000L; + static final int MERGE_SEGMENTS_LIMIT = 100; + static final int MAX_SEGMENTS_TO_MOVE = 100; + static final double PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE = 100; + static final int REPLICANT_LIFETIME = 15; + static final int REPLICATION_THROTTLE_LIMIT = 500; + static final int BALANCER_COMPUTE_THREADS = 1; + static final boolean EMIT_BALANCING_STATS = false; + static final boolean USE_BATCHED_SEGMENT_SAMPLER = true; + static final int MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 500; + static final int DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT = 70; + static final boolean PAUSE_COORDINATION = false; + static final boolean REPLICATE_AFTER_LOAD_TIMEOUT = false; + static final int MAX_NON_PRIMARY_REPLICANTS_TO_LOAD = Integer.MAX_VALUE; + static final boolean USE_ROUND_ROBIN_ASSIGNMENT = true; + static final boolean SMART_SEGMENT_LOADING = true; + } + public static class Builder { - private static final long DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS = - TimeUnit.MINUTES.toMillis(15); - private static final long DEFAULT_MERGE_BYTES_LIMIT = 524_288_000L; - private static final int DEFAULT_MERGE_SEGMENTS_LIMIT = 100; - private static final int DEFAULT_MAX_SEGMENTS_TO_MOVE = 100; - private static final double DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE = 100; - private static final int DEFAULT_REPLICANT_LIFETIME = 15; - private static final int DEFAULT_REPLICATION_THROTTLE_LIMIT = 500; - private static final int DEFAULT_BALANCER_COMPUTE_THREADS = 1; - private static final boolean DEFAULT_EMIT_BALANCING_STATS = false; - private static final boolean DEFAULT_USE_BATCHED_SEGMENT_SAMPLER = true; - private static final int DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 500; - private static final int DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT = 70; - private static final boolean DEFAULT_PAUSE_COORDINATION = false; - private static final boolean DEFAULT_REPLICATE_AFTER_LOAD_TIMEOUT = false; - private static final int DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD = Integer.MAX_VALUE; - private static final boolean DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT = true; - - private Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; + private Long markSegmentAsUnusedDelayMillis; private Long mergeBytesLimit; private Integer mergeSegmentsLimit; private Integer maxSegmentsToMove; @@ -546,11 +574,13 @@ public class CoordinatorDynamicConfig private Object dataSourcesToNotKillStalePendingSegmentsIn; private Integer maxSegmentsInNodeLoadingQueue; private Object decommissioningNodes; + private Map debugDimensions; private Integer decommissioningMaxPercentOfMaxSegmentsToMove; private Boolean pauseCoordination; private Boolean replicateAfterLoadTimeout; private Integer maxNonPrimaryReplicantsToLoad; private Boolean useRoundRobinSegmentAssignment; + private Boolean smartSegmentLoading; public Builder() { @@ -558,12 +588,12 @@ public class CoordinatorDynamicConfig @JsonCreator public Builder( - @JsonProperty("millisToWaitBeforeDeleting") - @Nullable Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, + @JsonProperty("millisToWaitBeforeDeleting") @Nullable Long markSegmentAsUnusedDelayMillis, @JsonProperty("mergeBytesLimit") @Nullable Long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") @Nullable Integer mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") @Nullable Integer maxSegmentsToMove, - @Deprecated @JsonProperty("percentOfSegmentsToConsiderPerMove") @Nullable Double percentOfSegmentsToConsiderPerMove, + @Deprecated @JsonProperty("percentOfSegmentsToConsiderPerMove") + @Nullable Double percentOfSegmentsToConsiderPerMove, @Deprecated @JsonProperty("useBatchedSegmentSampler") Boolean useBatchedSegmentSampler, @JsonProperty("replicantLifetime") @Nullable Integer replicantLifetime, @JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit, @@ -578,11 +608,12 @@ public class CoordinatorDynamicConfig @JsonProperty("pauseCoordination") @Nullable Boolean pauseCoordination, @JsonProperty("replicateAfterLoadTimeout") @Nullable Boolean replicateAfterLoadTimeout, @JsonProperty("maxNonPrimaryReplicantsToLoad") @Nullable Integer maxNonPrimaryReplicantsToLoad, - @JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment + @JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment, + @JsonProperty("smartSegmentLoading") @Nullable Boolean smartSegmentLoading, + @JsonProperty("debugDimensions") @Nullable Map debugDimensions ) { - this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = - leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; + this.markSegmentAsUnusedDelayMillis = markSegmentAsUnusedDelayMillis; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; @@ -601,11 +632,13 @@ public class CoordinatorDynamicConfig this.replicateAfterLoadTimeout = replicateAfterLoadTimeout; this.maxNonPrimaryReplicantsToLoad = maxNonPrimaryReplicantsToLoad; this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment; + this.smartSegmentLoading = smartSegmentLoading; + this.debugDimensions = debugDimensions; } - public Builder withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(long leadingTimeMillis) + public Builder withMarkSegmentAsUnusedDelayMillis(long leadingTimeMillis) { - this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = leadingTimeMillis; + this.markSegmentAsUnusedDelayMillis = leadingTimeMillis; return this; } @@ -641,6 +674,12 @@ public class CoordinatorDynamicConfig return this; } + public Builder withSmartSegmentLoading(boolean smartSegmentLoading) + { + this.smartSegmentLoading = smartSegmentLoading; + return this; + } + public Builder withReplicantLifetime(int replicantLifetime) { this.replicantLifetime = replicantLifetime; @@ -653,6 +692,12 @@ public class CoordinatorDynamicConfig return this; } + public Builder withDebugDimensions(Map debugDimensions) + { + this.debugDimensions = debugDimensions; + return this; + } + public Builder withBalancerComputeThreads(int balancerComputeThreads) { this.balancerComputeThreads = balancerComputeThreads; @@ -713,75 +758,78 @@ public class CoordinatorDynamicConfig return this; } + /** + * Builds a CoordinatoryDynamicConfig using either the configured values, or + * the default value if not configured. + */ public CoordinatorDynamicConfig build() { return new CoordinatorDynamicConfig( - leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null - ? DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS - : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, - mergeBytesLimit == null ? DEFAULT_MERGE_BYTES_LIMIT : mergeBytesLimit, - mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit, - maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove, - percentOfSegmentsToConsiderPerMove == null ? DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE - : percentOfSegmentsToConsiderPerMove, - useBatchedSegmentSampler == null ? DEFAULT_USE_BATCHED_SEGMENT_SAMPLER : useBatchedSegmentSampler, - replicantLifetime == null ? DEFAULT_REPLICANT_LIFETIME : replicantLifetime, - replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit, - balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads, - emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats, + valueOrDefault( + markSegmentAsUnusedDelayMillis, + Defaults.LEADING_MILLIS_BEFORE_MARK_UNUSED + ), + valueOrDefault(mergeBytesLimit, Defaults.MERGE_BYTES_LIMIT), + valueOrDefault(mergeSegmentsLimit, Defaults.MERGE_SEGMENTS_LIMIT), + valueOrDefault(maxSegmentsToMove, Defaults.MAX_SEGMENTS_TO_MOVE), + valueOrDefault(percentOfSegmentsToConsiderPerMove, Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE), + valueOrDefault(useBatchedSegmentSampler, Defaults.USE_BATCHED_SEGMENT_SAMPLER), + valueOrDefault(replicantLifetime, Defaults.REPLICANT_LIFETIME), + valueOrDefault(replicationThrottleLimit, Defaults.REPLICATION_THROTTLE_LIMIT), + valueOrDefault(balancerComputeThreads, Defaults.BALANCER_COMPUTE_THREADS), + valueOrDefault(emitBalancingStats, Defaults.EMIT_BALANCING_STATS), specificDataSourcesToKillUnusedSegmentsIn, dataSourcesToNotKillStalePendingSegmentsIn, - maxSegmentsInNodeLoadingQueue == null - ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE - : maxSegmentsInNodeLoadingQueue, + valueOrDefault(maxSegmentsInNodeLoadingQueue, Defaults.MAX_SEGMENTS_IN_NODE_LOADING_QUEUE), decommissioningNodes, - decommissioningMaxPercentOfMaxSegmentsToMove == null - ? DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT - : decommissioningMaxPercentOfMaxSegmentsToMove, - pauseCoordination == null ? DEFAULT_PAUSE_COORDINATION : pauseCoordination, - replicateAfterLoadTimeout == null ? DEFAULT_REPLICATE_AFTER_LOAD_TIMEOUT : replicateAfterLoadTimeout, - maxNonPrimaryReplicantsToLoad == null ? DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD - : maxNonPrimaryReplicantsToLoad, - useRoundRobinSegmentAssignment == null ? DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT : useRoundRobinSegmentAssignment + valueOrDefault( + decommissioningMaxPercentOfMaxSegmentsToMove, + Defaults.DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT + ), + valueOrDefault(pauseCoordination, Defaults.PAUSE_COORDINATION), + valueOrDefault(replicateAfterLoadTimeout, Defaults.REPLICATE_AFTER_LOAD_TIMEOUT), + valueOrDefault(maxNonPrimaryReplicantsToLoad, Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD), + valueOrDefault(useRoundRobinSegmentAssignment, Defaults.USE_ROUND_ROBIN_ASSIGNMENT), + valueOrDefault(smartSegmentLoading, Defaults.SMART_SEGMENT_LOADING), + debugDimensions ); } + private static T valueOrDefault(@Nullable T value, @NotNull T defaultValue) + { + return value == null ? defaultValue : value; + } + public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) { return new CoordinatorDynamicConfig( - leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null - ? defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() - : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, - mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit, - mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit, - maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove, - percentOfSegmentsToConsiderPerMove == null - ? defaults.getPercentOfSegmentsToConsiderPerMove() - : percentOfSegmentsToConsiderPerMove, - useBatchedSegmentSampler == null ? defaults.useBatchedSegmentSampler() : useBatchedSegmentSampler, - replicantLifetime == null ? defaults.getReplicantLifetime() : replicantLifetime, - replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit, - balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads, - emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats, - specificDataSourcesToKillUnusedSegmentsIn == null - ? defaults.getSpecificDataSourcesToKillUnusedSegmentsIn() - : specificDataSourcesToKillUnusedSegmentsIn, - dataSourcesToNotKillStalePendingSegmentsIn == null - ? defaults.getDataSourcesToNotKillStalePendingSegmentsIn() - : dataSourcesToNotKillStalePendingSegmentsIn, - maxSegmentsInNodeLoadingQueue == null - ? defaults.getMaxSegmentsInNodeLoadingQueue() - : maxSegmentsInNodeLoadingQueue, - decommissioningNodes == null ? defaults.getDecommissioningNodes() : decommissioningNodes, - decommissioningMaxPercentOfMaxSegmentsToMove == null - ? defaults.getDecommissioningMaxPercentOfMaxSegmentsToMove() - : decommissioningMaxPercentOfMaxSegmentsToMove, - pauseCoordination == null ? defaults.getPauseCoordination() : pauseCoordination, - replicateAfterLoadTimeout == null ? defaults.getReplicateAfterLoadTimeout() : replicateAfterLoadTimeout, - maxNonPrimaryReplicantsToLoad == null - ? defaults.getMaxNonPrimaryReplicantsToLoad() - : maxNonPrimaryReplicantsToLoad, - useRoundRobinSegmentAssignment == null ? defaults.isUseRoundRobinSegmentAssignment() : useRoundRobinSegmentAssignment + valueOrDefault( + markSegmentAsUnusedDelayMillis, + defaults.getMarkSegmentAsUnusedDelayMillis() + ), + valueOrDefault(mergeBytesLimit, defaults.getMergeBytesLimit()), + valueOrDefault(mergeSegmentsLimit, defaults.getMergeSegmentsLimit()), + valueOrDefault(maxSegmentsToMove, defaults.getMaxSegmentsToMove()), + valueOrDefault(percentOfSegmentsToConsiderPerMove, defaults.getPercentOfSegmentsToConsiderPerMove()), + valueOrDefault(useBatchedSegmentSampler, defaults.useBatchedSegmentSampler()), + valueOrDefault(replicantLifetime, defaults.getReplicantLifetime()), + valueOrDefault(replicationThrottleLimit, defaults.getReplicationThrottleLimit()), + valueOrDefault(balancerComputeThreads, defaults.getBalancerComputeThreads()), + valueOrDefault(emitBalancingStats, defaults.emitBalancingStats()), + valueOrDefault(specificDataSourcesToKillUnusedSegmentsIn, defaults.getSpecificDataSourcesToKillUnusedSegmentsIn()), + valueOrDefault(dataSourcesToNotKillStalePendingSegmentsIn, defaults.getDataSourcesToNotKillStalePendingSegmentsIn()), + valueOrDefault(maxSegmentsInNodeLoadingQueue, defaults.getMaxSegmentsInNodeLoadingQueue()), + valueOrDefault(decommissioningNodes, defaults.getDecommissioningNodes()), + valueOrDefault( + decommissioningMaxPercentOfMaxSegmentsToMove, + defaults.getDecommissioningMaxPercentOfMaxSegmentsToMove() + ), + valueOrDefault(pauseCoordination, defaults.getPauseCoordination()), + valueOrDefault(replicateAfterLoadTimeout, defaults.getReplicateAfterLoadTimeout()), + valueOrDefault(maxNonPrimaryReplicantsToLoad, defaults.getMaxNonPrimaryReplicantsToLoad()), + valueOrDefault(useRoundRobinSegmentAssignment, defaults.isUseRoundRobinSegmentAssignment()), + valueOrDefault(smartSegmentLoading, defaults.isSmartSegmentLoading()), + valueOrDefault(debugDimensions, defaults.getDebugDimensions()) ); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorStats.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorStats.java deleted file mode 100644 index 357cf45ae90..00000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorStats.java +++ /dev/null @@ -1,226 +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.druid.server.coordinator; - -import it.unimi.dsi.fastutil.objects.Object2LongMap; -import it.unimi.dsi.fastutil.objects.Object2LongMap.Entry; -import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.function.ObjLongConsumer; - -/** - */ -public class CoordinatorStats -{ - private final Map> perTierStats; - private final Map> perDataSourceStats; - private final Map> perDutyStats; - private final Object2LongOpenHashMap globalStats; - - public CoordinatorStats() - { - perTierStats = new HashMap<>(); - perDataSourceStats = new HashMap<>(); - perDutyStats = new HashMap<>(); - globalStats = new Object2LongOpenHashMap<>(); - } - - public boolean hasPerTierStats() - { - return !perTierStats.isEmpty(); - } - - public boolean hasPerDataSourceStats() - { - return !perDataSourceStats.isEmpty(); - } - - public boolean hasPerDutyStats() - { - return !perDutyStats.isEmpty(); - } - - public Set getTiers(final String statName) - { - final Object2LongOpenHashMap theStat = perTierStats.get(statName); - if (theStat == null) { - return Collections.emptySet(); - } - return Collections.unmodifiableSet(theStat.keySet()); - } - - public Set getDataSources(String statName) - { - final Object2LongOpenHashMap stat = perDataSourceStats.get(statName); - if (stat == null) { - return Collections.emptySet(); - } - return Collections.unmodifiableSet(stat.keySet()); - } - - public Set getDuties(String statName) - { - final Object2LongOpenHashMap stat = perDutyStats.get(statName); - if (stat == null) { - return Collections.emptySet(); - } - return Collections.unmodifiableSet(stat.keySet()); - } - - /** - * - * @param statName the name of the statistics - * @param tier the tier - * @return the value for the statistics {@code statName} under {@code tier} tier - * @throws NullPointerException if {@code statName} is not found - */ - public long getTieredStat(final String statName, final String tier) - { - return perTierStats.get(statName).getLong(tier); - } - - public void forEachTieredStat(final String statName, final ObjLongConsumer consumer) - { - final Object2LongOpenHashMap theStat = perTierStats.get(statName); - if (theStat != null) { - for (final Object2LongMap.Entry entry : theStat.object2LongEntrySet()) { - consumer.accept(entry.getKey(), entry.getLongValue()); - } - } - } - - public long getDataSourceStat(String statName, String dataSource) - { - return perDataSourceStats.get(statName).getLong(dataSource); - } - - public void forEachDataSourceStat(String statName, ObjLongConsumer consumer) - { - final Object2LongOpenHashMap stat = perDataSourceStats.get(statName); - if (stat != null) { - for (Entry entry : stat.object2LongEntrySet()) { - consumer.accept(entry.getKey(), entry.getLongValue()); - } - } - } - - public long getDutyStat(String statName, String duty) - { - return perDutyStats.get(statName).getLong(duty); - } - - public void forEachDutyStat(String statName, ObjLongConsumer consumer) - { - final Object2LongOpenHashMap stat = perDutyStats.get(statName); - if (stat != null) { - for (Entry entry : stat.object2LongEntrySet()) { - consumer.accept(entry.getKey(), entry.getLongValue()); - } - } - } - - public long getGlobalStat(final String statName) - { - return globalStats.getLong(statName); - } - - public void addToTieredStat(final String statName, final String tier, final long value) - { - perTierStats.computeIfAbsent(statName, ignored -> new Object2LongOpenHashMap<>()) - .addTo(tier, value); - } - - public void accumulateMaxTieredStat(final String statName, final String tier, final long value) - { - perTierStats.computeIfAbsent(statName, ignored -> new Object2LongOpenHashMap<>()) - .mergeLong(tier, value, Math::max); - } - - public void addToDataSourceStat(String statName, String dataSource, long value) - { - perDataSourceStats.computeIfAbsent(statName, k -> new Object2LongOpenHashMap<>()) - .addTo(dataSource, value); - } - - public void addToDutyStat(String statName, String duty, long value) - { - perDutyStats.computeIfAbsent(statName, k -> new Object2LongOpenHashMap<>()) - .addTo(duty, value); - } - - public void addToGlobalStat(final String statName, final long value) - { - globalStats.addTo(statName, value); - } - - public CoordinatorStats accumulate(final CoordinatorStats stats) - { - stats.perTierStats.forEach( - (final String statName, final Object2LongOpenHashMap urStat) -> { - - final Object2LongOpenHashMap myStat = perTierStats.computeIfAbsent( - statName, - name -> new Object2LongOpenHashMap<>() - ); - - for (final Object2LongMap.Entry entry : urStat.object2LongEntrySet()) { - myStat.addTo(entry.getKey(), entry.getLongValue()); - } - } - ); - - stats.perDataSourceStats.forEach( - (statName, urStat) -> { - final Object2LongOpenHashMap myStat = perDataSourceStats.computeIfAbsent( - statName, - k -> new Object2LongOpenHashMap<>() - ); - - for (Entry entry : urStat.object2LongEntrySet()) { - myStat.addTo(entry.getKey(), entry.getLongValue()); - } - } - ); - - stats.perDutyStats.forEach( - (statName, urStat) -> { - final Object2LongOpenHashMap myStat = perDutyStats.computeIfAbsent( - statName, - k -> new Object2LongOpenHashMap<>() - ); - - for (Entry entry : urStat.object2LongEntrySet()) { - myStat.addTo(entry.getKey(), entry.getLongValue()); - } - } - - ); - - for (final Object2LongMap.Entry entry : stats.globalStats.object2LongEntrySet()) { - globalStats.addTo(entry.getKey(), entry.getLongValue()); - } - - return this; - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java index 8fb4ccb056d..34cd830285d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java @@ -19,13 +19,11 @@ package org.apache.druid.server.coordinator; -import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.IAE; import org.apache.druid.utils.CollectionUtils; -import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -35,7 +33,6 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; import java.util.Set; -import java.util.TreeSet; /** * Contains a representation of the current state of the cluster by tier. @@ -43,83 +40,24 @@ import java.util.TreeSet; */ public class DruidCluster { - /** This static factory method must be called only from inside DruidClusterBuilder in tests. */ - @VisibleForTesting - static DruidCluster createDruidClusterFromBuilderInTest( - @Nullable Set realtimes, - Map> historicals, - @Nullable Set brokers - ) - { - return new DruidCluster(realtimes, historicals, brokers); - } + public static final DruidCluster EMPTY = builder().build(); private final Set realtimes; private final Map> historicals; private final Set brokers; - public DruidCluster() - { - this.realtimes = new HashSet<>(); - this.historicals = new HashMap<>(); - this.brokers = new HashSet<>(); - } - private DruidCluster( - @Nullable Set realtimes, - Map> historicals, - @Nullable Set brokers + Set realtimes, + Map> historicals, + Set brokers ) { - this.realtimes = realtimes == null ? new HashSet<>() : new HashSet<>(realtimes); + this.realtimes = Collections.unmodifiableSet(realtimes); this.historicals = CollectionUtils.mapValues( historicals, - holders -> CollectionUtils.newTreeSet(Comparator.reverseOrder(), holders) + holders -> CollectionUtils.newTreeSet(Comparator.naturalOrder(), holders) ); - this.brokers = brokers == null ? new HashSet<>() : new HashSet<>(brokers); - } - - public void add(ServerHolder serverHolder) - { - switch (serverHolder.getServer().getType()) { - case HISTORICAL: - addHistorical(serverHolder); - break; - case REALTIME: - addRealtime(serverHolder); - break; - case BRIDGE: - addHistorical(serverHolder); - break; - case INDEXER_EXECUTOR: - addRealtime(serverHolder); - break; - case BROKER: - addBroker(serverHolder); - break; - default: - throw new IAE("unknown server type[%s]", serverHolder.getServer().getType()); - } - } - - private void addRealtime(ServerHolder serverHolder) - { - realtimes.add(serverHolder); - } - - private void addHistorical(ServerHolder serverHolder) - { - final ImmutableDruidServer server = serverHolder.getServer(); - final NavigableSet tierServers = historicals.computeIfAbsent( - server.getTier(), - k -> new TreeSet<>(Collections.reverseOrder()) - ); - tierServers.add(serverHolder); - } - - private void addBroker(ServerHolder serverHolder) - { - brokers.add(serverHolder); + this.brokers = Collections.unmodifiableSet(brokers); } public Set getRealtimes() @@ -132,7 +70,6 @@ public class DruidCluster return historicals; } - public Set getBrokers() { return brokers; @@ -160,39 +97,71 @@ public class DruidCluster return allServers; } - public Iterable> getSortedHistoricalsByTier() - { - return historicals.values(); - } - public boolean isEmpty() { return historicals.isEmpty() && realtimes.isEmpty() && brokers.isEmpty(); } - public boolean hasHistoricals() + public static Builder builder() { - return !historicals.isEmpty(); + return new Builder(); } - public boolean hasRealtimes() + public static class Builder { - return !realtimes.isEmpty(); - } + private final Set realtimes = new HashSet<>(); + private final Map> historicals = new HashMap<>(); + private final Set brokers = new HashSet<>(); - public boolean hasBrokers() - { - return !brokers.isEmpty(); - } - - public boolean hasTier(String tier) - { - NavigableSet historicalServers = historicals.get(tier); - boolean historicalsHasTier = (historicalServers != null) && !historicalServers.isEmpty(); - if (historicalsHasTier) { - return true; + public Builder add(ServerHolder serverHolder) + { + switch (serverHolder.getServer().getType()) { + case BRIDGE: + case HISTORICAL: + addHistorical(serverHolder); + break; + case REALTIME: + case INDEXER_EXECUTOR: + realtimes.add(serverHolder); + break; + case BROKER: + brokers.add(serverHolder); + break; + default: + throw new IAE("unknown server type[%s]", serverHolder.getServer().getType()); + } + return this; } - return false; + public Builder addRealtimes(ServerHolder... realtimeServers) + { + realtimes.addAll(Arrays.asList(realtimeServers)); + return this; + } + + public Builder addBrokers(ServerHolder... brokers) + { + this.brokers.addAll(Arrays.asList(brokers)); + return this; + } + + public Builder addTier(String tier, ServerHolder... historicals) + { + this.historicals.computeIfAbsent(tier, t -> new HashSet<>()) + .addAll(Arrays.asList(historicals)); + return this; + } + + private void addHistorical(ServerHolder serverHolder) + { + final String tier = serverHolder.getServer().getTier(); + historicals.computeIfAbsent(tier, t -> new HashSet<>()).add(serverHolder); + } + + public DruidCluster build() + { + return new DruidCluster(realtimes, historicals, brokers); + } } + } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index e37ab05d6db..32c56bd0329 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -46,9 +46,6 @@ import org.apache.druid.guice.annotations.CoordinatorIndexingServiceDuty; import org.apache.druid.guice.annotations.CoordinatorMetadataStoreManagementDuty; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; @@ -60,22 +57,30 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.SegmentsMetadataManager; -import org.apache.druid.query.DruidMetrics; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordinator.balancer.BalancerStrategy; +import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; import org.apache.druid.server.coordinator.duty.BalanceSegments; +import org.apache.druid.server.coordinator.duty.CollectSegmentAndServerStats; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.duty.CompactionSegmentSearchPolicy; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroup; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups; import org.apache.druid.server.coordinator.duty.CoordinatorDuty; -import org.apache.druid.server.coordinator.duty.EmitClusterStatsAndMetrics; -import org.apache.druid.server.coordinator.duty.LogUsedSegments; -import org.apache.druid.server.coordinator.duty.MarkAsUnusedOvershadowedSegments; +import org.apache.druid.server.coordinator.duty.MarkOvershadowedSegmentsAsUnused; import org.apache.druid.server.coordinator.duty.RunRules; import org.apache.druid.server.coordinator.duty.UnloadUnusedSegments; -import org.apache.druid.server.coordinator.rules.LoadRule; -import org.apache.druid.server.coordinator.rules.Rule; -import org.apache.druid.server.initialization.jetty.ServiceUnavailableException; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.server.coordinator.loading.LoadQueueTaskMaster; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; +import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig; +import org.apache.druid.server.coordinator.loading.SegmentReplicaCount; +import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.CoordinatorStat; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.server.lookup.cache.LookupCoordinatorManager; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -86,16 +91,14 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Comparator; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; /** @@ -105,24 +108,21 @@ import java.util.stream.Collectors; public class DruidCoordinator { /** - * This comparator orders "freshest" segments first, i. e. segments with most recent intervals. - * - * It is used in historical nodes' {@link LoadQueuePeon}s to make historicals load more recent segment first. - * - * It is also used in {@link DruidCoordinatorRuntimeParams} for {@link - * DruidCoordinatorRuntimeParams#getUsedSegments()} - a collection of segments to be considered during some - * coordinator run for different {@link CoordinatorDuty}s. The order matters only for {@link - * RunRules}, which tries to apply the rules while iterating the segments in the order imposed by - * this comparator. In {@link LoadRule} the throttling limit may be hit (via {@link ReplicationThrottler}; see - * {@link CoordinatorDynamicConfig#getReplicationThrottleLimit()}). So before we potentially hit this limit, we want - * to schedule loading the more recent segments (among all of those that need to be loaded). - * - * In both {@link LoadQueuePeon}s and {@link RunRules}, we want to load more recent segments first - * because presumably they are queried more often and contain are more important data for users, so if the Druid - * cluster has availability problems and struggling to make all segments available immediately, at least we try to - * make more "important" (more recent) segments available as soon as possible. + * Orders newest segments (i.e. segments with most recent intervals) first. + * Used by: + *
    + *
  • {@link RunRules} duty to prioritize assignment of more recent segments. + * The order of segments matters because the {@link CoordinatorDynamicConfig#replicationThrottleLimit} + * might cause only a few segments to be picked for replication in a coordinator run. + *
  • + *
  • {@link LoadQueuePeon}s to prioritize load of more recent segments.
  • + *
+ * It is presumed that more recent segments are queried more often and contain + * more important data for users. This ordering thus ensures that if the cluster + * has availability or loading problems, the most recent segments are made + * available as soon as possible. */ - static final Comparator SEGMENT_COMPARATOR_RECENT_FIRST = Ordering + public static final Ordering SEGMENT_COMPARATOR_RECENT_FIRST = Ordering .from(Comparators.intervalsByEndThenStart()) .onResultOf(DataSegment::getInterval) .compound(Ordering.natural()) @@ -141,27 +141,32 @@ public class DruidCoordinator private final IndexingServiceClient indexingServiceClient; private final ScheduledExecutorService exec; private final LoadQueueTaskMaster taskMaster; - private final Map loadManagementPeons; + private final ConcurrentHashMap loadManagementPeons = new ConcurrentHashMap<>(); + private final SegmentLoadQueueManager loadQueueManager; private final ServiceAnnouncer serviceAnnouncer; private final DruidNode self; private final Set indexingServiceDuties; private final Set metadataStoreManagementDuties; private final CoordinatorCustomDutyGroups customDutyGroups; - private final BalancerStrategyFactory factory; + private final BalancerStrategyFactory balancerStrategyFactory; private final LookupCoordinatorManager lookupCoordinatorManager; private final DruidLeaderSelector coordLeaderSelector; private final CompactSegments compactSegments; private volatile boolean started = false; - private volatile SegmentReplicantLookup segmentReplicantLookup = null; /** - * Contains a map of segmentId to total replication factor across all tiers. This map is refreshed when load rules are - * evaluated. It is used by {@link DruidCoordinator} to supply this value to - * {@link org.apache.druid.server.http.MetadataResource}. + * Used to determine count of under-replicated or unavailable segments. + * Updated in each coordinator run in the {@link UpdateReplicationStatus} duty. + *

+ * This might have stale information if coordinator runs are delayed. But as + * long as the {@link SegmentsMetadataManager} has the latest information of + * used segments, we would only have false negatives and not false positives. + * In other words, we might report some segments as under-replicated or + * unavailable even if they are fully replicated. But if a segment is reported + * as fully replicated, it is guaranteed to be so. */ - private volatile Object2IntMap segmentIdToReplicationFactor = null; - private volatile DruidCluster cluster = null; + private volatile SegmentReplicationStatus segmentReplicationStatus = null; private int cachedBalancerThreadNumber; private ListeningExecutorService balancerExec; @@ -182,61 +187,17 @@ public class DruidCoordinator ScheduledExecutorFactory scheduledExecutorFactory, IndexingServiceClient indexingServiceClient, LoadQueueTaskMaster taskMaster, + SegmentLoadQueueManager loadQueueManager, ServiceAnnouncer serviceAnnouncer, @Self DruidNode self, @CoordinatorMetadataStoreManagementDuty Set metadataStoreManagementDuties, @CoordinatorIndexingServiceDuty Set indexingServiceDuties, CoordinatorCustomDutyGroups customDutyGroups, - BalancerStrategyFactory factory, + BalancerStrategyFactory balancerStrategyFactory, LookupCoordinatorManager lookupCoordinatorManager, @Coordinator DruidLeaderSelector coordLeaderSelector, CompactionSegmentSearchPolicy compactionSegmentSearchPolicy ) - { - this( - config, - configManager, - segmentsMetadataManager, - serverInventoryView, - metadataRuleManager, - emitter, - scheduledExecutorFactory, - indexingServiceClient, - taskMaster, - serviceAnnouncer, - self, - new ConcurrentHashMap<>(), - indexingServiceDuties, - metadataStoreManagementDuties, - customDutyGroups, - factory, - lookupCoordinatorManager, - coordLeaderSelector, - compactionSegmentSearchPolicy - ); - } - - DruidCoordinator( - DruidCoordinatorConfig config, - JacksonConfigManager configManager, - SegmentsMetadataManager segmentsMetadataManager, - ServerInventoryView serverInventoryView, - MetadataRuleManager metadataRuleManager, - ServiceEmitter emitter, - ScheduledExecutorFactory scheduledExecutorFactory, - IndexingServiceClient indexingServiceClient, - LoadQueueTaskMaster taskMaster, - ServiceAnnouncer serviceAnnouncer, - DruidNode self, - ConcurrentMap loadQueuePeonMap, - Set indexingServiceDuties, - Set metadataStoreManagementDuties, - CoordinatorCustomDutyGroups customDutyGroups, - BalancerStrategyFactory factory, - LookupCoordinatorManager lookupCoordinatorManager, - DruidLeaderSelector coordLeaderSelector, - CompactionSegmentSearchPolicy compactionSegmentSearchPolicy - ) { this.config = config; this.configManager = configManager; @@ -255,11 +216,11 @@ public class DruidCoordinator this.exec = scheduledExecutorFactory.create(1, "Coordinator-Exec--%d"); - this.loadManagementPeons = loadQueuePeonMap; - this.factory = factory; + this.balancerStrategyFactory = balancerStrategyFactory; this.lookupCoordinatorManager = lookupCoordinatorManager; this.coordLeaderSelector = coordLeaderSelector; this.compactSegments = initializeCompactSegmentsDuty(compactionSegmentSearchPolicy); + this.loadQueueManager = loadQueueManager; } public boolean isLeader() @@ -272,76 +233,42 @@ public class DruidCoordinator return loadManagementPeons; } - /** - * @return tier -> { dataSource -> underReplicationCount } map - */ - public Map> computeUnderReplicationCountsPerDataSourcePerTier() + public Map> getTierToDatasourceToUnderReplicatedCount(boolean useClusterView) { final Iterable dataSegments = segmentsMetadataManager.iterateAllUsedSegments(); - return computeUnderReplicationCountsPerDataSourcePerTierForSegmentsInternal(dataSegments, false); + return computeUnderReplicated(dataSegments, useClusterView); } - /** - * @return tier -> { dataSource -> underReplicationCount } map - */ - public Map> computeUnderReplicationCountsPerDataSourcePerTierUsingClusterView() - { - final Iterable dataSegments = segmentsMetadataManager.iterateAllUsedSegments(); - return computeUnderReplicationCountsPerDataSourcePerTierForSegmentsInternal(dataSegments, true); - } - - /** - * segmentReplicantLookup use in this method could potentially be stale since it is only updated on coordinator runs. - * However, this is ok as long as the {@param dataSegments} is refreshed/latest as this would at least still ensure - * that the stale data in segmentReplicantLookup would be under counting replication levels, - * rather than potentially falsely reporting that everything is available. - * - * @return tier -> { dataSource -> underReplicationCount } map - */ - public Map> computeUnderReplicationCountsPerDataSourcePerTierForSegments( - Iterable dataSegments + public Map> getTierToDatasourceToUnderReplicatedCount( + Iterable dataSegments, + boolean useClusterView ) { - return computeUnderReplicationCountsPerDataSourcePerTierForSegmentsInternal(dataSegments, false); + return computeUnderReplicated(dataSegments, useClusterView); } - /** - * segmentReplicantLookup or cluster use in this method could potentially be stale since it is only updated on coordinator runs. - * However, this is ok as long as the {@param dataSegments} is refreshed/latest as this would at least still ensure - * that the stale data in segmentReplicantLookup and cluster would be under counting replication levels, - * rather than potentially falsely reporting that everything is available. - * - * @return tier -> { dataSource -> underReplicationCount } map - */ - public Map> computeUnderReplicationCountsPerDataSourcePerTierForSegmentsUsingClusterView( - Iterable dataSegments - ) + public Object2IntMap getDatasourceToUnavailableSegmentCount() { - return computeUnderReplicationCountsPerDataSourcePerTierForSegmentsInternal(dataSegments, true); - } - - public Object2IntMap computeNumsUnavailableUsedSegmentsPerDataSource() - { - if (segmentReplicantLookup == null) { + if (segmentReplicationStatus == null) { return Object2IntMaps.emptyMap(); } - final Object2IntOpenHashMap numsUnavailableUsedSegmentsPerDataSource = new Object2IntOpenHashMap<>(); + final Object2IntOpenHashMap datasourceToUnavailableSegments = new Object2IntOpenHashMap<>(); final Iterable dataSegments = segmentsMetadataManager.iterateAllUsedSegments(); - for (DataSegment segment : dataSegments) { - if (segmentReplicantLookup.getLoadedReplicants(segment.getId()) == 0) { - numsUnavailableUsedSegmentsPerDataSource.addTo(segment.getDataSource(), 1); + SegmentReplicaCount replicaCount = segmentReplicationStatus.getReplicaCountsInCluster(segment.getId()); + if (replicaCount != null && replicaCount.totalLoaded() > 0) { + datasourceToUnavailableSegments.addTo(segment.getDataSource(), 0); } else { - numsUnavailableUsedSegmentsPerDataSource.addTo(segment.getDataSource(), 0); + datasourceToUnavailableSegments.addTo(segment.getDataSource(), 1); } } - return numsUnavailableUsedSegmentsPerDataSource; + return datasourceToUnavailableSegments; } - public Map getLoadStatus() + public Map getDatasourceToLoadStatus() { final Map loadStatus = new HashMap<>(); final Collection dataSources = @@ -356,7 +283,7 @@ public class DruidCoordinator final DruidDataSource loadedView = druidServer.getDataSource(dataSource.getName()); if (loadedView != null) { // This does not use segments.removeAll(loadedView.getSegments()) for performance reasons. - // Please see https://github.com/apache/druid/pull/5632 and LoadStatusBenchmark for more info. + // Please see https://github.com/apache/druid/pull/5632 for more info. for (DataSegment serverSegment : loadedView.getSegments()) { segments.remove(serverSegment); } @@ -365,13 +292,23 @@ public class DruidCoordinator final int numUnavailableSegments = segments.size(); loadStatus.put( dataSource.getName(), - 100 * ((double) (numPublishedSegments - numUnavailableSegments) / (double) numPublishedSegments) + (numPublishedSegments - numUnavailableSegments) * 100.0 / numPublishedSegments ); } return loadStatus; } + @Nullable + public Integer getReplicationFactor(SegmentId segmentId) + { + if (segmentReplicationStatus != null) { + return segmentReplicationStatus.getReplicaCountsInCluster(segmentId).required(); + } else { + return null; + } + } + @Nullable public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) { @@ -389,27 +326,21 @@ public class DruidCoordinator return compactSegments.getAutoCompactionSnapshot(); } - public CoordinatorDynamicConfig getDynamicConfigs() + private CoordinatorDynamicConfig getDynamicConfigs() { return CoordinatorDynamicConfig.current(configManager); } - public CoordinatorCompactionConfig getCompactionConfig() + private CoordinatorCompactionConfig getCompactionConfig() { return CoordinatorCompactionConfig.current(configManager); } public void markSegmentsAsUnused(String datasource, Set segmentIds) { - log.debug("Marking [%d] segments of datasource [%s] as unused: %s", segmentIds.size(), datasource, segmentIds); + log.debug("Marking [%d] segments of datasource [%s] as unused.", segmentIds.size(), datasource); int updatedCount = segmentsMetadataManager.markSegmentsAsUnused(segmentIds); - log.info("Successfully marked [%d] segments of datasource [%s] as unused", updatedCount, datasource); - } - - public void markSegmentAsUnused(DataSegment segment) - { - log.debug("Marking segment[%s] as unused", segment.getId()); - segmentsMetadataManager.markSegmentAsUnused(segment.getId()); + log.info("Successfully marked [%d] segments of datasource [%s] as unused.", updatedCount, datasource); } public String getCurrentLeader() @@ -417,111 +348,6 @@ public class DruidCoordinator return coordLeaderSelector.getCurrentLeader(); } - public void moveSegment( - DruidCoordinatorRuntimeParams params, - ImmutableDruidServer fromServer, - ImmutableDruidServer toServer, - DataSegment segment, - final LoadPeonCallback callback - ) - { - if (segment == null) { - log.makeAlert(new IAE("Can not move null DataSegment"), "Exception moving null segment").emit(); - if (callback != null) { - callback.execute(false); - } - throw new ISE("Cannot move null DataSegment"); - } - SegmentId segmentId = segment.getId(); - try { - if (fromServer.getMetadata().equals(toServer.getMetadata())) { - throw new IAE("Cannot move [%s] to and from the same server [%s]", segmentId, fromServer.getName()); - } - - ImmutableDruidDataSource dataSource = params.getDataSourcesSnapshot().getDataSource(segment.getDataSource()); - if (dataSource == null) { - throw new IAE("Unable to find dataSource for segment [%s] in metadata", segmentId); - } - - // get segment information from SegmentsMetadataManager instead of getting it from fromServer's. - // This is useful when SegmentsMetadataManager and fromServer DataSegment's are different for same - // identifier (say loadSpec differs because of deep storage migration). - final DataSegment segmentToLoad = dataSource.getSegment(segment.getId()); - if (segmentToLoad == null) { - throw new IAE("No segment metadata found for segment Id [%s]", segment.getId()); - } - final LoadQueuePeon loadPeon = loadManagementPeons.get(toServer.getName()); - if (loadPeon == null) { - throw new IAE("LoadQueuePeon hasn't been created yet for path [%s]", toServer.getName()); - } - - final LoadQueuePeon dropPeon = loadManagementPeons.get(fromServer.getName()); - if (dropPeon == null) { - throw new IAE("LoadQueuePeon hasn't been created yet for path [%s]", fromServer.getName()); - } - - final ServerHolder toHolder = new ServerHolder(toServer, loadPeon); - if (toHolder.getAvailableSize() < segmentToLoad.getSize()) { - throw new IAE( - "Not enough capacity on server [%s] for segment [%s]. Required: %,d, available: %,d.", - toServer.getName(), - segmentToLoad, - segmentToLoad.getSize(), - toHolder.getAvailableSize() - ); - } - - final LoadPeonCallback loadPeonCallback = success -> { - dropPeon.unmarkSegmentToDrop(segmentToLoad); - if (callback != null) { - callback.execute(success); - } - }; - - // mark segment to drop before it is actually loaded on server - // to be able to account this information in DruidBalancerStrategy immediately - dropPeon.markSegmentToDrop(segmentToLoad); - try { - loadPeon.loadSegment( - segmentToLoad, - success -> { - // Drop segment only if: - // (1) segment load was successful on toServer - // AND (2) segment not already queued for drop on fromServer - // AND (3a) loading is http-based - // OR (3b) inventory shows segment loaded on toServer - - // Do not check the inventory with http loading as the HTTP - // response is enough to determine load success or failure - try { - if (success - && !dropPeon.getSegmentsToDrop().contains(segment) - && (taskMaster.isHttpLoading() - || serverInventoryView.isSegmentLoadedByServer(toServer.getName(), segment))) { - dropPeon.dropSegment(segment, loadPeonCallback); - } else { - loadPeonCallback.execute(success); - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - ); - } - catch (Exception e) { - dropPeon.unmarkSegmentToDrop(segmentToLoad); - throw new RuntimeException(e); - } - } - catch (Exception e) { - log.makeAlert(e, "Exception moving segment %s", segmentId).emit(); - if (callback != null) { - callback.execute(false); - } - } - } - @VisibleForTesting public int getCachedBalancerThreadNumber() { @@ -585,60 +411,25 @@ public class DruidCoordinator public void runCompactSegmentsDuty() { final int startingLeaderCounter = coordLeaderSelector.localTerm(); - DutiesRunnable compactSegmentsDuty = new DutiesRunnable(makeCompactSegmentsDuty(), startingLeaderCounter, COMPACT_SEGMENTS_DUTIES_DUTY_GROUP); + DutiesRunnable compactSegmentsDuty = new DutiesRunnable( + makeCompactSegmentsDuty(), + startingLeaderCounter, + COMPACT_SEGMENTS_DUTIES_DUTY_GROUP, + null + ); compactSegmentsDuty.run(); } - private Map> computeUnderReplicationCountsPerDataSourcePerTierForSegmentsInternal( + private Map> computeUnderReplicated( Iterable dataSegments, boolean computeUsingClusterView ) { - final Map> underReplicationCountsPerDataSourcePerTier = new HashMap<>(); - - if (segmentReplicantLookup == null) { - return underReplicationCountsPerDataSourcePerTier; + if (segmentReplicationStatus == null) { + return Collections.emptyMap(); + } else { + return segmentReplicationStatus.getTierToDatasourceToUnderReplicated(dataSegments, !computeUsingClusterView); } - - if (computeUsingClusterView && cluster == null) { - throw new ServiceUnavailableException( - "coordinator hasn't populated information about cluster yet, try again later"); - } - - final DateTime now = DateTimes.nowUtc(); - - for (final DataSegment segment : dataSegments) { - final List rules = metadataRuleManager.getRulesWithDefault(segment.getDataSource()); - - for (final Rule rule : rules) { - if (!rule.appliesTo(segment, now)) { - // Rule did not match. Continue to the next Rule. - continue; - } - if (!rule.canLoadSegments()) { - // Rule matched but rule does not and cannot load segments. - // Hence, there is no need to update underReplicationCountsPerDataSourcePerTier map - break; - } - - if (computeUsingClusterView) { - rule.updateUnderReplicatedWithClusterView( - underReplicationCountsPerDataSourcePerTier, - segmentReplicantLookup, - cluster, - segment - ); - } else { - rule.updateUnderReplicated(underReplicationCountsPerDataSourcePerTier, segmentReplicantLookup, segment); - } - - // Only the first matching rule applies. This is because the Coordinator cycle through all used segments - // and match each segment with the first rule that applies. Each segment may only match a single rule. - break; - } - } - - return underReplicationCountsPerDataSourcePerTier; } private void becomeLeader() @@ -659,43 +450,52 @@ public class DruidCoordinator serviceAnnouncer.announce(self); final int startingLeaderCounter = coordLeaderSelector.localTerm(); - final List> dutiesRunnables = new ArrayList<>(); + final List dutiesRunnables = new ArrayList<>(); dutiesRunnables.add( - Pair.of( - new DutiesRunnable(makeHistoricalManagementDuties(), startingLeaderCounter, HISTORICAL_MANAGEMENT_DUTIES_DUTY_GROUP), + new DutiesRunnable( + makeHistoricalManagementDuties(), + startingLeaderCounter, + HISTORICAL_MANAGEMENT_DUTIES_DUTY_GROUP, config.getCoordinatorPeriod() ) ); if (indexingServiceClient != null) { dutiesRunnables.add( - Pair.of( - new DutiesRunnable(makeIndexingServiceDuties(), startingLeaderCounter, INDEXING_SERVICE_DUTIES_DUTY_GROUP), + new DutiesRunnable( + makeIndexingServiceDuties(), + startingLeaderCounter, + INDEXING_SERVICE_DUTIES_DUTY_GROUP, config.getCoordinatorIndexingPeriod() ) ); } dutiesRunnables.add( - Pair.of( - new DutiesRunnable(makeMetadataStoreManagementDuties(), startingLeaderCounter, METADATA_STORE_MANAGEMENT_DUTIES_DUTY_GROUP), + new DutiesRunnable( + makeMetadataStoreManagementDuties(), + startingLeaderCounter, + METADATA_STORE_MANAGEMENT_DUTIES_DUTY_GROUP, config.getCoordinatorMetadataStoreManagementPeriod() ) ); for (CoordinatorCustomDutyGroup customDutyGroup : customDutyGroups.getCoordinatorCustomDutyGroups()) { dutiesRunnables.add( - Pair.of( - new DutiesRunnable(customDutyGroup.getCustomDutyList(), startingLeaderCounter, customDutyGroup.getName()), + new DutiesRunnable( + customDutyGroup.getCustomDutyList(), + startingLeaderCounter, + customDutyGroup.getName(), customDutyGroup.getPeriod() ) ); log.info( - "Done making custom coordinator duties %s for group %s", - customDutyGroup.getCustomDutyList().stream().map(duty -> duty.getClass().getName()).collect(Collectors.toList()), + "Done making custom coordinator duties [%s] for group [%s]", + customDutyGroup.getCustomDutyList().stream() + .map(duty -> duty.getClass().getName()).collect(Collectors.toList()), customDutyGroup.getName() ); } - for (final Pair dutiesRunnable : dutiesRunnables) { + for (final DutiesRunnable dutiesRunnable : dutiesRunnables) { // CompactSegmentsDuty can takes a non trival amount of time to complete. // Hence, we schedule at fixed rate to make sure the other tasks still run at approximately every // config.getCoordinatorIndexingPeriod() period. Note that cautious should be taken @@ -703,23 +503,19 @@ public class DruidCoordinator ScheduledExecutors.scheduleAtFixedRate( exec, config.getCoordinatorStartDelay(), - dutiesRunnable.rhs, - new Callable() - { - private final DutiesRunnable theRunnable = dutiesRunnable.lhs; + dutiesRunnable.getPeriod(), + () -> { + if (coordLeaderSelector.isLeader() + && startingLeaderCounter == coordLeaderSelector.localTerm()) { + dutiesRunnable.run(); + } - @Override - public ScheduledExecutors.Signal call() - { - if (coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) { - theRunnable.run(); - } - if (coordLeaderSelector.isLeader() - && startingLeaderCounter == coordLeaderSelector.localTerm()) { // (We might no longer be leader) - return ScheduledExecutors.Signal.REPEAT; - } else { - return ScheduledExecutors.Signal.STOP; - } + // Check if we are still leader before re-scheduling + if (coordLeaderSelector.isLeader() + && startingLeaderCounter == coordLeaderSelector.localTerm()) { + return ScheduledExecutors.Signal.REPEAT; + } else { + return ScheduledExecutors.Signal.STOP; } } ); @@ -751,31 +547,56 @@ public class DruidCoordinator } } + @VisibleForTesting + protected void initBalancerExecutor() + { + final int currentNumber = getDynamicConfigs().getBalancerComputeThreads(); + + if (balancerExec == null) { + balancerExec = createNewBalancerExecutor(currentNumber); + } else if (cachedBalancerThreadNumber != currentNumber) { + log.info( + "balancerComputeThreads has changed from [%d] to [%d], recreating the thread pool.", + cachedBalancerThreadNumber, + currentNumber + ); + balancerExec.shutdownNow(); + balancerExec = createNewBalancerExecutor(currentNumber); + } + } + + private ListeningExecutorService createNewBalancerExecutor(int numThreads) + { + cachedBalancerThreadNumber = numThreads; + return MoreExecutors.listeningDecorator( + Execs.multiThreaded(numThreads, "coordinator-cost-balancer-%s") + ); + } + private List makeHistoricalManagementDuties() { return ImmutableList.of( - new LogUsedSegments(), new UpdateCoordinatorStateAndPrepareCluster(), - new RunRules(DruidCoordinator.this), - new UnloadUnusedSegments(), - new MarkAsUnusedOvershadowedSegments(DruidCoordinator.this), - new BalanceSegments(DruidCoordinator.this) + new RunRules(), + new UpdateReplicationStatus(), + new UnloadUnusedSegments(loadQueueManager), + new MarkOvershadowedSegmentsAsUnused(DruidCoordinator.this), + new BalanceSegments(), + new CollectSegmentAndServerStats(DruidCoordinator.this) ); } @VisibleForTesting List makeIndexingServiceDuties() { - List duties = new ArrayList<>(); - duties.add(new LogUsedSegments()); - duties.addAll(indexingServiceDuties); + final List duties = new ArrayList<>(indexingServiceDuties); // CompactSegmentsDuty should be the last duty as it can take a long time to complete // We do not have to add compactSegments if it is already enabled in the custom duty group if (getCompactSegmentsDutyFromCustomGroups().isEmpty()) { duties.addAll(makeCompactSegmentsDuty()); } log.debug( - "Done making indexing service duties %s", + "Initialized indexing service duties [%s].", duties.stream().map(duty -> duty.getClass().getName()).collect(Collectors.toList()) ); return ImmutableList.copyOf(duties); @@ -783,15 +604,12 @@ public class DruidCoordinator private List makeMetadataStoreManagementDuties() { - List duties = ImmutableList.builder() - .addAll(metadataStoreManagementDuties) - .build(); - + List duties = ImmutableList.copyOf(metadataStoreManagementDuties); log.debug( - "Done making metadata store management duties %s", + "Initialized metadata store management duties [%s].", duties.stream().map(duty -> duty.getClass().getName()).collect(Collectors.toList()) ); - return ImmutableList.copyOf(duties); + return duties; } @VisibleForTesting @@ -802,7 +620,10 @@ public class DruidCoordinator return new CompactSegments(config, compactionSegmentSearchPolicy, indexingServiceClient); } else { if (compactSegmentsDutyFromCustomGroups.size() > 1) { - log.warn("More than one compactSegments duty is configured in the Coordinator Custom Duty Group. The first duty will be picked up."); + log.warn( + "More than one compactSegments duty is configured in the Coordinator Custom Duty Group." + + " The first duty will be picked up." + ); } return compactSegmentsDutyFromCustomGroups.get(0); } @@ -813,7 +634,8 @@ public class DruidCoordinator { return customDutyGroups.getCoordinatorCustomDutyGroups() .stream() - .flatMap(coordinatorCustomDutyGroup -> coordinatorCustomDutyGroup.getCustomDutyList().stream()) + .flatMap(coordinatorCustomDutyGroup -> + coordinatorCustomDutyGroup.getCustomDutyList().stream()) .filter(duty -> duty instanceof CompactSegments) .map(duty -> (CompactSegments) duty) .collect(Collectors.toList()); @@ -824,72 +646,34 @@ public class DruidCoordinator return ImmutableList.of(compactSegments); } - @Nullable - public Integer getReplicationFactorForSegment(SegmentId segmentId) + private class DutiesRunnable implements Runnable { - return segmentIdToReplicationFactor == null ? null : segmentIdToReplicationFactor.getInt(segmentId); - } - - @VisibleForTesting - protected class DutiesRunnable implements Runnable - { - private final long startTimeNanos = System.nanoTime(); + private final DateTime coordinatorStartTime = DateTimes.nowUtc(); private final List duties; private final int startingLeaderCounter; - private final String dutiesRunnableAlias; + private final String dutyGroupName; + private final Duration period; - protected DutiesRunnable(List duties, final int startingLeaderCounter, String alias) + DutiesRunnable( + List duties, + final int startingLeaderCounter, + String alias, + Duration period + ) { - // Automatically add EmitClusterStatsAndMetrics duty to the group if it does not already exists - // This is to avoid human coding error (forgetting to add the EmitClusterStatsAndMetrics duty to the group) - // causing metrics from the duties to not being emitted. - if (duties.stream().noneMatch(duty -> duty instanceof EmitClusterStatsAndMetrics)) { - boolean isContainCompactSegmentDuty = duties.stream().anyMatch(duty -> duty instanceof CompactSegments); - List allDuties = new ArrayList<>(duties); - allDuties.add(new EmitClusterStatsAndMetrics(DruidCoordinator.this, alias, isContainCompactSegmentDuty)); - this.duties = allDuties; - } else { - this.duties = duties; - } + this.duties = duties; this.startingLeaderCounter = startingLeaderCounter; - this.dutiesRunnableAlias = alias; - } - - @VisibleForTesting - protected void initBalancerExecutor() - { - final int currentNumber = getDynamicConfigs().getBalancerComputeThreads(); - final String threadNameFormat = "coordinator-cost-balancer-%s"; - // fist time initialization - if (balancerExec == null) { - balancerExec = MoreExecutors.listeningDecorator(Execs.multiThreaded( - currentNumber, - threadNameFormat - )); - cachedBalancerThreadNumber = currentNumber; - return; - } - - if (cachedBalancerThreadNumber != currentNumber) { - log.info( - "balancerComputeThreads has been changed from [%s] to [%s], recreating the thread pool.", - cachedBalancerThreadNumber, - currentNumber - ); - balancerExec.shutdownNow(); - balancerExec = MoreExecutors.listeningDecorator(Execs.multiThreaded( - currentNumber, - threadNameFormat - )); - cachedBalancerThreadNumber = currentNumber; - } + this.dutyGroupName = alias; + this.period = period; } @Override public void run() { try { - final long globalStart = System.nanoTime(); + log.info("Starting coordinator run for group [%s]", dutyGroupName); + final long globalStart = System.currentTimeMillis(); + synchronized (lock) { if (!coordLeaderSelector.isLeader()) { log.info("LEGGO MY EGGO. [%s] is leader.", coordLeaderSelector.getCurrentLeader()); @@ -910,32 +694,29 @@ public class DruidCoordinator } } - initBalancerExecutor(); - BalancerStrategy balancerStrategy = factory.createBalancerStrategy(balancerExec); - // Do coordinator stuff. DataSourcesSnapshot dataSourcesSnapshot = segmentsMetadataManager.getSnapshotOfDataSourcesWithAllUsedSegments(); DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams - .newBuilder() + .newBuilder(coordinatorStartTime) .withDatabaseRuleManager(metadataRuleManager) - .withStartTimeNanos(startTimeNanos) .withSnapshotOfDataSourcesWithAllUsedSegments(dataSourcesSnapshot) .withDynamicConfigs(getDynamicConfigs()) .withCompactionConfig(getCompactionConfig()) .withEmitter(emitter) - .withBalancerStrategy(balancerStrategy) .build(); + log.info( + "Initialized run params for group [%s] with [%,d] used segments in [%d] datasources.", + dutyGroupName, params.getUsedSegments().size(), dataSourcesSnapshot.getDataSourcesMap().size() + ); boolean coordinationPaused = getDynamicConfigs().getPauseCoordination(); if (coordinationPaused && coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) { - log.debug( - "Coordination is paused via dynamic configs! I will not be running Coordination Duties at this time" - ); + log.info("Coordination has been paused. Duties will not run until coordination is resumed."); } for (CoordinatorDuty duty : duties) { @@ -944,57 +725,89 @@ public class DruidCoordinator && coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) { - final long start = System.nanoTime(); + final long start = System.currentTimeMillis(); params = duty.run(params); - final long end = System.nanoTime(); + final long end = System.currentTimeMillis(); + final String dutyName = duty.getClass().getName(); if (params == null) { - // This duty wanted to cancel the run. No log message, since the duty should have logged a reason. + log.info("Stopping run for group [%s] on request of duty [%s].", dutyGroupName, dutyName); return; } else { - params.getCoordinatorStats().addToDutyStat("runtime", duty.getClass().getName(), TimeUnit.NANOSECONDS.toMillis(end - start)); + final RowKey rowKey = RowKey.builder().add(Dimension.DUTY, dutyName).build(); + params.getCoordinatorStats().add(Stats.CoordinatorRun.DUTY_RUN_TIME, rowKey, end - start); } } } - // Update the immutable replication factor map with latest values. - // This value is set here as it is recalculated during load rule evaluation. - if (params.getSegmentReplicantLookup() != null) { - segmentIdToReplicationFactor = params.getSegmentReplicantLookup().getSegmentIdToReplicationFactor(); + // Emit stats collected from all duties + final CoordinatorRunStats allStats = params.getCoordinatorStats(); + if (allStats.rowCount() > 0) { + final AtomicInteger emittedCount = new AtomicInteger(); + allStats.forEachStat( + (dimensionValues, stat, value) -> { + if (stat.shouldEmit()) { + emitStat(stat, dimensionValues, value); + emittedCount.incrementAndGet(); + } + } + ); + + log.info( + "Emitted [%d] stats for group [%s]. All collected stats:%s\n", + emittedCount.get(), dutyGroupName, allStats.buildStatsTable() + ); } // Emit the runtime of the full DutiesRunnable - params.getEmitter().emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, dutiesRunnableAlias) - .build("coordinator/global/time", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - globalStart)) - ); + final long runMillis = System.currentTimeMillis() - globalStart; + emitStat(Stats.CoordinatorRun.GROUP_RUN_TIME, Collections.emptyMap(), runMillis); + log.info("Finished coordinator run for group [%s] in [%d] ms", dutyGroupName, runMillis); } catch (Exception e) { log.makeAlert(e, "Caught exception, ignoring so that schedule keeps going.").emit(); } } - @VisibleForTesting - public List getDuties() + private void emitStat(CoordinatorStat stat, Map dimensionValues, long value) { - return duties; + if (stat.equals(Stats.Balancer.NORMALIZED_COST_X_1000)) { + value = value / 1000; + } + + ServiceMetricEvent.Builder eventBuilder = new ServiceMetricEvent.Builder() + .setDimension(Dimension.DUTY_GROUP.reportedName(), dutyGroupName); + dimensionValues.forEach( + (dim, dimValue) -> eventBuilder.setDimension(dim.reportedName(), dimValue) + ); + emitter.emit(eventBuilder.build(stat.getMetricName(), value)); + } + + Duration getPeriod() + { + return period; } @Override public String toString() { - return "DutiesRunnable{" + - "dutiesRunnableAlias='" + dutiesRunnableAlias + '\'' + - '}'; + return "DutiesRunnable{group='" + dutyGroupName + '\'' + '}'; } } /** - * Updates the enclosing {@link DruidCoordinator}'s state and prepares an immutable view of the cluster state (which - * consists of {@link DruidCluster} and {@link SegmentReplicantLookup}) and feeds it into {@link - * DruidCoordinatorRuntimeParams} for use in subsequent {@link CoordinatorDuty}s (see the order in {@link - * #makeHistoricalManagementDuties()}). + * This duty does the following: + *

    + *
  • Prepares an immutable {@link DruidCluster} consisting of {@link ServerHolder}s + * which represent the current state of the servers in the cluster.
  • + *
  • Starts and stops load peons for new and disappeared servers respectively.
  • + *
  • Cancels in-progress loads on all decommissioning servers. This is done + * here to ensure that under-replicated segments are assigned to active servers + * in the {@link RunRules} duty after this.
  • + *
  • Initializes the {@link BalancerStrategy} for the run.
  • + *
+ * + * @see #makeHistoricalManagementDuties() for the order of duties */ private class UpdateCoordinatorStateAndPrepareCluster implements CoordinatorDuty { @@ -1005,28 +818,62 @@ public class DruidCoordinator List currentServers = prepareCurrentServers(); startPeonsForNewServers(currentServers); - - cluster = prepareCluster(params, currentServers); - segmentReplicantLookup = SegmentReplicantLookup.make(cluster, getDynamicConfigs().getReplicateAfterLoadTimeout()); - stopPeonsForDisappearedServers(currentServers); - final RoundRobinServerSelector roundRobinServerSelector; - if (params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) { - roundRobinServerSelector = new RoundRobinServerSelector(cluster); - log.info("Using round-robin segment assignment."); - } else { - roundRobinServerSelector = null; - } + final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig(); + final SegmentLoadingConfig segmentLoadingConfig = params.getSegmentLoadingConfig(); + + final DruidCluster cluster = prepareCluster(dynamicConfig, segmentLoadingConfig, currentServers); + cancelLoadsOnDecommissioningServers(cluster); + + initBalancerExecutor(); + final BalancerStrategy balancerStrategy = balancerStrategyFactory.createBalancerStrategy(balancerExec); + log.info( + "Using balancer strategy [%s] with round-robin assignment [%s] and debug dimensions [%s].", + balancerStrategy.getClass().getSimpleName(), + segmentLoadingConfig.isUseRoundRobinSegmentAssignment(), + dynamicConfig.getDebugDimensions() + ); return params.buildFromExisting() .withDruidCluster(cluster) - .withLoadManagementPeons(loadManagementPeons) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withRoundRobinServerSelector(roundRobinServerSelector) + .withBalancerStrategy(balancerStrategy) + .withSegmentAssignerUsing(loadQueueManager) .build(); } + /** + * Cancels all load/move operations on decommissioning servers. This should + * be done before initializing the SegmentReplicantLookup so that + * under-replicated segments can be assigned in the current run itself. + */ + private void cancelLoadsOnDecommissioningServers(DruidCluster cluster) + { + final AtomicInteger cancelledCount = new AtomicInteger(0); + final List decommissioningServers + = cluster.getAllServers().stream() + .filter(ServerHolder::isDecommissioning) + .collect(Collectors.toList()); + + for (ServerHolder server : decommissioningServers) { + server.getQueuedSegments().forEach( + (segment, action) -> { + // Cancel the operation if it is a type of load + if (action.isLoad() && server.cancelOperation(action, segment)) { + cancelledCount.incrementAndGet(); + } + } + ); + } + + if (cancelledCount.get() > 0) { + log.info( + "Cancelled [%d] load/move operations on [%d] decommissioning servers.", + cancelledCount.get(), decommissioningServers.size() + ); + } + } + List prepareCurrentServers() { List currentServers = serverInventoryView @@ -1062,21 +909,26 @@ public class DruidCoordinator } } - DruidCluster prepareCluster(DruidCoordinatorRuntimeParams params, List currentServers) + DruidCluster prepareCluster( + CoordinatorDynamicConfig dynamicConfig, + SegmentLoadingConfig segmentLoadingConfig, + List currentServers + ) { - Set decommissioningServers = params.getCoordinatorDynamicConfig().getDecommissioningNodes(); - final DruidCluster cluster = new DruidCluster(); + final Set decommissioningServers = dynamicConfig.getDecommissioningNodes(); + final DruidCluster.Builder cluster = DruidCluster.builder(); for (ImmutableDruidServer server : currentServers) { cluster.add( new ServerHolder( server, loadManagementPeons.get(server.getName()), decommissioningServers.contains(server.getHost()), - params.getCoordinatorDynamicConfig().getMaxSegmentsInNodeLoadingQueue() + segmentLoadingConfig.getMaxSegmentsInLoadQueue(), + segmentLoadingConfig.getMaxLifetimeInLoadQueue() ) ); } - return cluster; + return cluster.build(); } void stopPeonsForDisappearedServers(List servers) @@ -1092,5 +944,22 @@ public class DruidCoordinator } } } + + /** + * Updates replication status of all used segments. This duty must run after + * {@link RunRules} so that the number of required replicas for all segments + * has been determined. + */ + private class UpdateReplicationStatus implements CoordinatorDuty + { + + @Override + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) + { + segmentReplicationStatus = params.getSegmentReplicationStatus(); + return params; + } + + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index e3703144ebd..ed1c57dbb19 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -25,18 +25,24 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; +import org.apache.druid.server.coordinator.balancer.BalancerStrategy; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; +import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig; +import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus; +import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; +import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; +import java.util.Collections; import java.util.Map; import java.util.Set; import java.util.TreeSet; -import java.util.concurrent.TimeUnit; /** */ @@ -56,60 +62,54 @@ public class DruidCoordinatorRuntimeParams return segmentsSet; } - private final long startTimeNanos; + private final DateTime coordinatorStartTime; private final DruidCluster druidCluster; private final MetadataRuleManager databaseRuleManager; - private final SegmentReplicantLookup segmentReplicantLookup; + private final StrategicSegmentAssigner segmentAssigner; private final @Nullable TreeSet usedSegments; private final @Nullable DataSourcesSnapshot dataSourcesSnapshot; - private final Map loadManagementPeons; - private final ReplicationThrottler replicationManager; private final ServiceEmitter emitter; private final CoordinatorDynamicConfig coordinatorDynamicConfig; private final CoordinatorCompactionConfig coordinatorCompactionConfig; - private final CoordinatorStats stats; + private final SegmentLoadingConfig segmentLoadingConfig; + private final CoordinatorRunStats stats; private final BalancerStrategy balancerStrategy; private final Set broadcastDatasources; - private final @Nullable RoundRobinServerSelector roundRobinServerSelector; private DruidCoordinatorRuntimeParams( - long startTimeNanos, + DateTime coordinatorStartTime, DruidCluster druidCluster, MetadataRuleManager databaseRuleManager, - SegmentReplicantLookup segmentReplicantLookup, + StrategicSegmentAssigner segmentAssigner, @Nullable TreeSet usedSegments, @Nullable DataSourcesSnapshot dataSourcesSnapshot, - Map loadManagementPeons, - ReplicationThrottler replicationManager, - @Nullable RoundRobinServerSelector roundRobinServerSelector, ServiceEmitter emitter, CoordinatorDynamicConfig coordinatorDynamicConfig, CoordinatorCompactionConfig coordinatorCompactionConfig, - CoordinatorStats stats, + SegmentLoadingConfig segmentLoadingConfig, + CoordinatorRunStats stats, BalancerStrategy balancerStrategy, Set broadcastDatasources ) { - this.startTimeNanos = startTimeNanos; + this.coordinatorStartTime = coordinatorStartTime; this.druidCluster = druidCluster; this.databaseRuleManager = databaseRuleManager; - this.segmentReplicantLookup = segmentReplicantLookup; + this.segmentAssigner = segmentAssigner; this.usedSegments = usedSegments; this.dataSourcesSnapshot = dataSourcesSnapshot; - this.loadManagementPeons = loadManagementPeons; - this.replicationManager = replicationManager; - this.roundRobinServerSelector = roundRobinServerSelector; this.emitter = emitter; this.coordinatorDynamicConfig = coordinatorDynamicConfig; this.coordinatorCompactionConfig = coordinatorCompactionConfig; + this.segmentLoadingConfig = segmentLoadingConfig; this.stats = stats; this.balancerStrategy = balancerStrategy; this.broadcastDatasources = broadcastDatasources; } - public long getStartTimeNanos() + public DateTime getCoordinatorStartTime() { - return startTimeNanos; + return coordinatorStartTime; } public DruidCluster getDruidCluster() @@ -122,15 +122,17 @@ public class DruidCoordinatorRuntimeParams return databaseRuleManager; } - public SegmentReplicantLookup getSegmentReplicantLookup() + @Nullable + public SegmentReplicationStatus getSegmentReplicationStatus() { - return segmentReplicantLookup; + return segmentAssigner == null ? null : segmentAssigner.getReplicationStatus(); + } + + public StrategicSegmentAssigner getSegmentAssigner() + { + return segmentAssigner; } - /** - * Creates and returns a "dataSource -> VersionedIntervalTimeline[version String, DataSegment]" map with "used" - * segments. - */ public Map getUsedSegmentsTimelinesPerDataSource() { Preconditions.checkState(dataSourcesSnapshot != null, "dataSourcesSnapshot or usedSegments must be set"); @@ -143,22 +145,6 @@ public class DruidCoordinatorRuntimeParams return usedSegments; } - public Map getLoadManagementPeons() - { - return loadManagementPeons; - } - - public ReplicationThrottler getReplicationManager() - { - return replicationManager; - } - - @Nullable - public RoundRobinServerSelector getRoundRobinServerSelector() - { - return roundRobinServerSelector; - } - public ServiceEmitter getEmitter() { return emitter; @@ -174,7 +160,12 @@ public class DruidCoordinatorRuntimeParams return coordinatorCompactionConfig; } - public CoordinatorStats getCoordinatorStats() + public SegmentLoadingConfig getSegmentLoadingConfig() + { + return segmentLoadingConfig; + } + + public CoordinatorRunStats getCoordinatorStats() { return stats; } @@ -189,62 +180,30 @@ public class DruidCoordinatorRuntimeParams return broadcastDatasources; } - public boolean coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements() - { - long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos(); - long lagNanos = TimeUnit.MILLISECONDS.toNanos( - coordinatorDynamicConfig.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() - ); - return nanosElapsedSinceCoordinatorStart > lagNanos; - } - public DataSourcesSnapshot getDataSourcesSnapshot() { Preconditions.checkState(dataSourcesSnapshot != null, "usedSegments or dataSourcesSnapshot must be set"); return dataSourcesSnapshot; } - public static Builder newBuilder() + public static Builder newBuilder(DateTime coordinatorStartTime) { - return new Builder(); + return new Builder(coordinatorStartTime); } public Builder buildFromExisting() { return new Builder( - startTimeNanos, + coordinatorStartTime, druidCluster, databaseRuleManager, - segmentReplicantLookup, + segmentAssigner, usedSegments, dataSourcesSnapshot, - loadManagementPeons, - replicationManager, - roundRobinServerSelector, - emitter, - coordinatorDynamicConfig, - coordinatorCompactionConfig, - stats, - balancerStrategy, - broadcastDatasources - ); - } - - public Builder buildFromExistingWithoutSegmentsMetadata() - { - return new Builder( - startTimeNanos, - druidCluster, - databaseRuleManager, - segmentReplicantLookup, - null, // usedSegments - null, // dataSourcesSnapshot - loadManagementPeons, - replicationManager, - roundRobinServerSelector, emitter, coordinatorDynamicConfig, coordinatorCompactionConfig, + segmentLoadingConfig, stats, balancerStrategy, broadcastDatasources @@ -253,70 +212,55 @@ public class DruidCoordinatorRuntimeParams public static class Builder { - private @Nullable Long startTimeNanos; + private final DateTime coordinatorStartTime; private DruidCluster druidCluster; private MetadataRuleManager databaseRuleManager; - private SegmentReplicantLookup segmentReplicantLookup; + private SegmentLoadQueueManager loadQueueManager; + private StrategicSegmentAssigner segmentAssigner; private @Nullable TreeSet usedSegments; private @Nullable DataSourcesSnapshot dataSourcesSnapshot; - private final Map loadManagementPeons; - private ReplicationThrottler replicationManager; - private @Nullable RoundRobinServerSelector roundRobinServerSelector; private ServiceEmitter emitter; private CoordinatorDynamicConfig coordinatorDynamicConfig; private CoordinatorCompactionConfig coordinatorCompactionConfig; - private CoordinatorStats stats; + private SegmentLoadingConfig segmentLoadingConfig; + private CoordinatorRunStats stats; private BalancerStrategy balancerStrategy; private Set broadcastDatasources; - private Builder() + private Builder(DateTime coordinatorStartTime) { - this.startTimeNanos = null; - this.druidCluster = null; - this.databaseRuleManager = null; - this.segmentReplicantLookup = null; - this.usedSegments = null; - this.dataSourcesSnapshot = null; - this.loadManagementPeons = new HashMap<>(); - this.replicationManager = null; - this.roundRobinServerSelector = null; - this.emitter = null; - this.stats = new CoordinatorStats(); + this.coordinatorStartTime = coordinatorStartTime; this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build(); this.coordinatorCompactionConfig = CoordinatorCompactionConfig.empty(); - this.broadcastDatasources = new HashSet<>(); + this.broadcastDatasources = Collections.emptySet(); } - Builder( - long startTimeNanos, + private Builder( + DateTime coordinatorStartTime, DruidCluster cluster, MetadataRuleManager databaseRuleManager, - SegmentReplicantLookup segmentReplicantLookup, + StrategicSegmentAssigner segmentAssigner, @Nullable TreeSet usedSegments, @Nullable DataSourcesSnapshot dataSourcesSnapshot, - Map loadManagementPeons, - ReplicationThrottler replicationManager, - @Nullable RoundRobinServerSelector roundRobinServerSelector, ServiceEmitter emitter, CoordinatorDynamicConfig coordinatorDynamicConfig, CoordinatorCompactionConfig coordinatorCompactionConfig, - CoordinatorStats stats, + SegmentLoadingConfig segmentLoadingConfig, + CoordinatorRunStats stats, BalancerStrategy balancerStrategy, Set broadcastDatasources ) { - this.startTimeNanos = startTimeNanos; + this.coordinatorStartTime = coordinatorStartTime; this.druidCluster = cluster; this.databaseRuleManager = databaseRuleManager; - this.segmentReplicantLookup = segmentReplicantLookup; + this.segmentAssigner = segmentAssigner; this.usedSegments = usedSegments; this.dataSourcesSnapshot = dataSourcesSnapshot; - this.loadManagementPeons = loadManagementPeons; - this.replicationManager = replicationManager; - this.roundRobinServerSelector = roundRobinServerSelector; this.emitter = emitter; this.coordinatorDynamicConfig = coordinatorDynamicConfig; this.coordinatorCompactionConfig = coordinatorCompactionConfig; + this.segmentLoadingConfig = segmentLoadingConfig; this.stats = stats; this.balancerStrategy = balancerStrategy; this.broadcastDatasources = broadcastDatasources; @@ -324,42 +268,60 @@ public class DruidCoordinatorRuntimeParams public DruidCoordinatorRuntimeParams build() { - Preconditions.checkNotNull(startTimeNanos, "startTime must be set"); + initStatsIfRequired(); + initSegmentLoadingConfigIfRequired(); + initSegmentAssignerIfRequired(); + return new DruidCoordinatorRuntimeParams( - startTimeNanos, + coordinatorStartTime, druidCluster, databaseRuleManager, - segmentReplicantLookup, + segmentAssigner, usedSegments, dataSourcesSnapshot, - loadManagementPeons, - replicationManager, - getOrCreateRoundRobinServerSelector(), emitter, coordinatorDynamicConfig, coordinatorCompactionConfig, + segmentLoadingConfig, stats, balancerStrategy, broadcastDatasources ); } - private RoundRobinServerSelector getOrCreateRoundRobinServerSelector() + private void initStatsIfRequired() { - if (druidCluster == null || coordinatorDynamicConfig == null - || !coordinatorDynamicConfig.isUseRoundRobinSegmentAssignment()) { - return null; - } else if (roundRobinServerSelector == null) { - return new RoundRobinServerSelector(druidCluster); - } else { - return roundRobinServerSelector; + Map debugDimensions = + coordinatorDynamicConfig == null ? null : coordinatorDynamicConfig.getValidatedDebugDimensions(); + stats = stats == null ? new CoordinatorRunStats(debugDimensions) : stats; + } + + private void initSegmentLoadingConfigIfRequired() + { + if (segmentLoadingConfig == null + && coordinatorDynamicConfig != null + && usedSegments != null) { + segmentLoadingConfig = SegmentLoadingConfig.create(coordinatorDynamicConfig, usedSegments.size()); } } - public Builder withStartTimeNanos(long startTimeNanos) + private void initSegmentAssignerIfRequired() { - this.startTimeNanos = startTimeNanos; - return this; + if (segmentAssigner != null || loadQueueManager == null) { + return; + } + + Preconditions.checkNotNull(druidCluster); + Preconditions.checkNotNull(balancerStrategy); + Preconditions.checkNotNull(segmentLoadingConfig); + Preconditions.checkNotNull(stats); + segmentAssigner = new StrategicSegmentAssigner( + loadQueueManager, + druidCluster, + balancerStrategy, + segmentLoadingConfig, + stats + ); } public Builder withDruidCluster(DruidCluster cluster) @@ -374,9 +336,13 @@ public class DruidCoordinatorRuntimeParams return this; } - public Builder withSegmentReplicantLookup(SegmentReplicantLookup lookup) + /** + * Sets the {@link SegmentLoadQueueManager} which is used to construct the + * {@link StrategicSegmentAssigner} for this run. + */ + public Builder withSegmentAssignerUsing(SegmentLoadQueueManager loadQueueManager) { - this.segmentReplicantLookup = lookup; + this.loadQueueManager = loadQueueManager; return this; } @@ -403,50 +369,12 @@ public class DruidCoordinatorRuntimeParams return this; } - /** This method must be used in test code only. */ - @VisibleForTesting - public Builder withUsedSegmentsTimelinesPerDataSourceInTest( - Map usedSegmentsTimelinesPerDataSource - ) - { - this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegmentsTimelines( - usedSegmentsTimelinesPerDataSource, - ImmutableMap.of() - ); - usedSegments = createUsedSegmentsSet(dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()); - return this; - } - - public Builder withLoadManagementPeons(Map loadManagementPeonsCollection) - { - loadManagementPeons.putAll(loadManagementPeonsCollection); - return this; - } - - public Builder withReplicationManager(ReplicationThrottler replicationManager) - { - this.replicationManager = replicationManager; - return this; - } - - public Builder withRoundRobinServerSelector(RoundRobinServerSelector roundRobinServerSelector) - { - this.roundRobinServerSelector = roundRobinServerSelector; - return this; - } - public Builder withEmitter(ServiceEmitter emitter) { this.emitter = emitter; return this; } - public Builder withCoordinatorStats(CoordinatorStats stats) - { - this.stats.accumulate(stats); - return this; - } - public Builder withDynamicConfigs(CoordinatorDynamicConfig configs) { this.coordinatorDynamicConfig = configs; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java b/server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java deleted file mode 100644 index 4e25086fd74..00000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java +++ /dev/null @@ -1,182 +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.druid.server.coordinator; - -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.timeline.SegmentId; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -/** - * The ReplicationThrottler is used to throttle the number of replicants that are created. - */ -public class ReplicationThrottler -{ - private static final EmittingLogger log = new EmittingLogger(ReplicationThrottler.class); - - private final Map replicatingLookup = new HashMap<>(); - private final ReplicatorSegmentHolder currentlyReplicating = new ReplicatorSegmentHolder(); - - private volatile int maxReplicants; - private volatile int maxLifetime; - private volatile boolean loadPrimaryReplicantsOnly; - - public ReplicationThrottler(int maxReplicants, int maxLifetime, boolean loadPrimaryReplicantsOnly) - { - updateParams(maxReplicants, maxLifetime, loadPrimaryReplicantsOnly); - } - - public void updateParams(int maxReplicants, int maxLifetime, boolean loadPrimaryReplicantsOnly) - { - this.maxReplicants = maxReplicants; - this.maxLifetime = maxLifetime; - this.loadPrimaryReplicantsOnly = loadPrimaryReplicantsOnly; - } - - public void updateReplicationState(String tier) - { - update(tier, currentlyReplicating, replicatingLookup, "create"); - } - - public boolean isLoadPrimaryReplicantsOnly() - { - return loadPrimaryReplicantsOnly; - } - - public void setLoadPrimaryReplicantsOnly(boolean loadPrimaryReplicantsOnly) - { - this.loadPrimaryReplicantsOnly = loadPrimaryReplicantsOnly; - } - - private void update(String tier, ReplicatorSegmentHolder holder, Map lookup, String type) - { - int size = holder.getNumProcessing(tier); - if (size != 0) { - log.info( - "[%s]: Replicant %s queue still has %d segments. Lifetime[%d]. Segments %s", - tier, - type, - size, - holder.getLifetime(tier), - holder.getCurrentlyProcessingSegmentsAndHosts(tier) - ); - holder.reduceLifetime(tier); - lookup.put(tier, false); - - if (holder.getLifetime(tier) < 0) { - log.makeAlert("[%s]: Replicant %s queue stuck after %d+ runs!", tier, type, maxLifetime) - .addData("segments", holder.getCurrentlyProcessingSegmentsAndHosts(tier)) - .emit(); - } - } else { - log.info("[%s]: Replicant %s queue is empty.", tier, type); - lookup.put(tier, true); - holder.resetLifetime(tier); - } - } - - public boolean canCreateReplicant(String tier) - { - return !loadPrimaryReplicantsOnly && replicatingLookup.get(tier) && !currentlyReplicating.isAtMaxReplicants(tier); - } - - public void registerReplicantCreation(String tier, SegmentId segmentId, String serverId) - { - currentlyReplicating.addSegment(tier, segmentId, serverId); - } - - public void unregisterReplicantCreation(String tier, SegmentId segmentId) - { - currentlyReplicating.removeSegment(tier, segmentId); - } - - private class ReplicatorSegmentHolder - { - private final Map> currentlyProcessingSegments = new HashMap<>(); - private final Map lifetimes = new HashMap<>(); - - public boolean isAtMaxReplicants(String tier) - { - final ConcurrentHashMap segments = currentlyProcessingSegments.get(tier); - return (segments != null && segments.size() >= maxReplicants); - } - - public void addSegment(String tier, SegmentId segmentId, String serverId) - { - ConcurrentHashMap segments = - currentlyProcessingSegments.computeIfAbsent(tier, t -> new ConcurrentHashMap<>()); - - if (!isAtMaxReplicants(tier)) { - segments.put(segmentId, serverId); - } - } - - public void removeSegment(String tier, SegmentId segmentId) - { - ConcurrentMap segments = currentlyProcessingSegments.get(tier); - if (segments != null) { - segments.remove(segmentId); - } - } - - public int getNumProcessing(String tier) - { - ConcurrentMap segments = currentlyProcessingSegments.get(tier); - return (segments == null) ? 0 : segments.size(); - } - - public int getLifetime(String tier) - { - Integer lifetime = lifetimes.putIfAbsent(tier, maxLifetime); - return lifetime != null ? lifetime : maxLifetime; - } - - public void reduceLifetime(String tier) - { - lifetimes.compute( - tier, - (t, lifetime) -> { - if (lifetime == null) { - return maxLifetime - 1; - } - return lifetime - 1; - } - ); - } - - public void resetLifetime(String tier) - { - lifetimes.put(tier, maxLifetime); - } - - public List getCurrentlyProcessingSegmentsAndHosts(String tier) - { - ConcurrentMap segments = currentlyProcessingSegments.get(tier); - List segmentsAndHosts = new ArrayList<>(); - segments.forEach((segmentId, serverId) -> segmentsAndHosts.add(segmentId + " ON " + serverId)); - return segmentsAndHosts; - } - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java b/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java deleted file mode 100644 index 96e4fe81276..00000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java +++ /dev/null @@ -1,174 +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.druid.server.coordinator; - -import com.google.common.collect.HashBasedTable; -import com.google.common.collect.Table; -import it.unimi.dsi.fastutil.objects.Object2IntMap; -import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; -import it.unimi.dsi.fastutil.objects.Object2LongMap; -import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; -import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; - -import java.util.HashMap; -import java.util.Map; -import java.util.SortedSet; - -/** - * A lookup for the number of replicants of a given segment for a certain tier. - */ -public class SegmentReplicantLookup -{ - public static SegmentReplicantLookup make(DruidCluster cluster, boolean replicateAfterLoadTimeout) - { - final Table segmentsInCluster = HashBasedTable.create(); - - /** - * For each tier, this stores the number of replicants for all the segments presently queued to load in {@link cluster}. - * Segments that have failed to load due to the load timeout may not be present in this table if {@link replicateAfterLoadTimeout} is true. - * This is to enable additional replication of the timed out segments for improved availability. - */ - final Table loadingSegments = HashBasedTable.create(); - - for (SortedSet serversByType : cluster.getSortedHistoricalsByTier()) { - for (ServerHolder serverHolder : serversByType) { - ImmutableDruidServer server = serverHolder.getServer(); - - for (DataSegment segment : server.iterateAllSegments()) { - Integer numReplicants = segmentsInCluster.get(segment.getId(), server.getTier()); - if (numReplicants == null) { - numReplicants = 0; - } - segmentsInCluster.put(segment.getId(), server.getTier(), numReplicants + 1); - } - - // Also account for queued segments - for (DataSegment segment : serverHolder.getPeon().getSegmentsToLoad()) { - Integer numReplicants = loadingSegments.get(segment.getId(), server.getTier()); - if (numReplicants == null) { - numReplicants = 0; - } - // Timed out segments need to be replicated in another server for faster availability. - // Therefore we skip incrementing numReplicants for timed out segments if replicateAfterLoadTimeout is enabled. - if (!replicateAfterLoadTimeout || !serverHolder.getPeon().getTimedOutSegments().contains(segment)) { - loadingSegments.put(segment.getId(), server.getTier(), numReplicants + 1); - } - } - } - } - - return new SegmentReplicantLookup(segmentsInCluster, loadingSegments, cluster); - } - - private final Table segmentsInCluster; - private final Table loadingSegments; - private final Map segmentIdToReplicationFactor = new HashMap<>(); - private final DruidCluster cluster; - - private SegmentReplicantLookup( - Table segmentsInCluster, - Table loadingSegments, - DruidCluster cluster - ) - { - this.segmentsInCluster = segmentsInCluster; - this.loadingSegments = loadingSegments; - this.cluster = cluster; - } - - public Map getClusterTiers(SegmentId segmentId) - { - Map retVal = segmentsInCluster.row(segmentId); - return (retVal == null) ? new HashMap<>() : retVal; - } - - int getLoadedReplicants(SegmentId segmentId) - { - Map allTiers = segmentsInCluster.row(segmentId); - int retVal = 0; - for (Integer replicants : allTiers.values()) { - retVal += replicants; - } - return retVal; - } - - public int getLoadedReplicants(SegmentId segmentId, String tier) - { - Integer retVal = segmentsInCluster.get(segmentId, tier); - return (retVal == null) ? 0 : retVal; - } - - // TODO: Refactor this setter, as this class is following a singleton pattern with only getters, and this breaks convention. - // This would be revamped in https://github.com/apache/druid/pull/13197 - public void setReplicationFactor(SegmentId segmentId, Integer requiredReplicas) - { - segmentIdToReplicationFactor.put(segmentId, requiredReplicas); - } - - public Object2IntMap getSegmentIdToReplicationFactor() - { - return new Object2IntOpenHashMap<>(segmentIdToReplicationFactor); - } - - private int getLoadingReplicants(SegmentId segmentId, String tier) - { - Integer retVal = loadingSegments.get(segmentId, tier); - return (retVal == null) ? 0 : retVal; - } - - private int getLoadingReplicants(SegmentId segmentId) - { - Map allTiers = loadingSegments.row(segmentId); - int retVal = 0; - for (Integer replicants : allTiers.values()) { - retVal += replicants; - } - return retVal; - } - - public int getTotalReplicants(SegmentId segmentId) - { - return getLoadedReplicants(segmentId) + getLoadingReplicants(segmentId); - } - - public int getTotalReplicants(SegmentId segmentId, String tier) - { - return getLoadedReplicants(segmentId, tier) + getLoadingReplicants(segmentId, tier); - } - - public Object2LongMap getBroadcastUnderReplication(SegmentId segmentId) - { - Object2LongOpenHashMap perTier = new Object2LongOpenHashMap<>(); - for (ServerHolder holder : cluster.getAllServers()) { - // Only record tier entry for server that is segment broadcast target - if (holder.getServer().getType().isSegmentBroadcastTarget()) { - // Every broadcast target server should be serving 1 replica of the segment - if (!holder.isServingSegment(segmentId)) { - perTier.addTo(holder.getServer().getTier(), 1L); - } else { - perTier.putIfAbsent(holder.getServer().getTier(), 0); - } - } - } - return perTier; - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java index 057dfb118c8..79f2c38ab20 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java @@ -20,43 +20,156 @@ package org.apache.druid.server.coordinator; import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.server.coordinator.loading.SegmentAction; +import org.apache.druid.server.coordinator.loading.SegmentHolder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; /** + * Encapsulates the state of a DruidServer during a single coordinator run. + *

+ * ServerHolders are naturally ordered by available size, servers with more + * available size first. */ public class ServerHolder implements Comparable { - private static final Logger log = new Logger(ServerHolder.class); + private static final Comparator MORE_AVAILABLE_SIZE_SERVER_FIRST = + Comparator.comparing(ServerHolder::getAvailableSize) + .thenComparing(holder -> holder.getServer().getHost()) + .thenComparing(holder -> holder.getServer().getTier()) + .thenComparing(holder -> holder.getServer().getType()) + .reversed(); + + private static final EmittingLogger log = new EmittingLogger(ServerHolder.class); + private final ImmutableDruidServer server; private final LoadQueuePeon peon; private final boolean isDecommissioning; - private final int maxSegmentsInLoadQueue; + private final int maxAssignmentsInRun; + private final int maxLifetimeInQueue; + + private final int movingSegmentCount; + private final int loadingReplicaCount; + + private int totalAssignmentsInRun; + private long sizeOfLoadingSegments; + private long sizeOfDroppingSegments; + + /** + * Remove entries from this map only if the operation is cancelled. + * Do not remove entries on load/drop success or failure during the run. + */ + private final Map queuedSegments = new HashMap<>(); + + /** + * Segments that are expected to be loaded on this server once all the + * operations in progress have completed. + */ + private final Set projectedSegments = new HashSet<>(); public ServerHolder(ImmutableDruidServer server, LoadQueuePeon peon) { - this(server, peon, false); + this(server, peon, false, 0, 1); } public ServerHolder(ImmutableDruidServer server, LoadQueuePeon peon, boolean isDecommissioning) { - this(server, peon, isDecommissioning, 0); + this(server, peon, isDecommissioning, 0, 1); } + /** + * Creates a new ServerHolder valid for a single coordinator run. + * + * @param server Underlying Druid server + * @param peon Load queue peon for this server + * @param isDecommissioning Whether the server is decommissioning + * @param maxSegmentsInLoadQueue Max number of segments that can be present in + * the load queue at any point. If this is 0, the + * load queue can have an unlimited number of segments. + * @param maxLifetimeInQueue Number of coordinator runs after a which a segment + * in load/drop queue is considered to be stuck. + */ public ServerHolder( ImmutableDruidServer server, LoadQueuePeon peon, boolean isDecommissioning, - int maxSegmentsInNodeLoadingQueue + int maxSegmentsInLoadQueue, + int maxLifetimeInQueue ) { this.server = server; this.peon = peon; this.isDecommissioning = isDecommissioning; - this.maxSegmentsInLoadQueue = maxSegmentsInNodeLoadingQueue; + + this.maxAssignmentsInRun = maxSegmentsInLoadQueue == 0 + ? Integer.MAX_VALUE + : maxSegmentsInLoadQueue - peon.getSegmentsToLoad().size(); + this.maxLifetimeInQueue = maxLifetimeInQueue; + + final AtomicInteger movingSegmentCount = new AtomicInteger(); + final AtomicInteger loadingReplicaCount = new AtomicInteger(); + initializeQueuedSegments(movingSegmentCount, loadingReplicaCount); + + this.movingSegmentCount = movingSegmentCount.get(); + this.loadingReplicaCount = loadingReplicaCount.get(); + } + + private void initializeQueuedSegments( + AtomicInteger movingSegmentCount, + AtomicInteger loadingReplicaCount + ) + { + projectedSegments.addAll(server.iterateAllSegments()); + + final List expiredSegments = new ArrayList<>(); + peon.getSegmentsInQueue().forEach( + (holder) -> { + int runsInQueue = holder.incrementAndGetRunsInQueue(); + if (runsInQueue > maxLifetimeInQueue) { + expiredSegments.add(holder); + } + + final SegmentAction action = holder.getAction(); + addToQueuedSegments(holder.getSegment(), simplify(action)); + + if (action == SegmentAction.MOVE_TO) { + movingSegmentCount.incrementAndGet(); + } + if (action == SegmentAction.REPLICATE) { + loadingReplicaCount.incrementAndGet(); + } + } + ); + + peon.getSegmentsMarkedToDrop().forEach( + segment -> addToQueuedSegments(segment, SegmentAction.MOVE_FROM) + ); + + if (!expiredSegments.isEmpty()) { + List expiredSegmentsSubList = + expiredSegments.size() > 10 ? expiredSegments.subList(0, 10) : expiredSegments; + + log.makeAlert( + "Load queue for server [%s], tier [%s] has [%d] segments stuck.", + server.getName(), server.getTier(), expiredSegments.size() + ) + .addData("segments", expiredSegmentsSubList.toString()) + .addData("maxLifetime", maxLifetimeInQueue).emit(); + } } public ImmutableDruidServer getServer() @@ -74,19 +187,9 @@ public class ServerHolder implements Comparable return server.getMaxSize(); } - public long getCurrServerSize() - { - return server.getCurrSize(); - } - - public long getLoadQueueSize() - { - return peon.getLoadQueueSize(); - } - public long getSizeUsed() { - return getCurrServerSize() + getLoadQueueSize(); + return server.getCurrSize() + sizeOfLoadingSegments - sizeOfDroppingSegments; } public double getPercentUsed() @@ -94,13 +197,6 @@ public class ServerHolder implements Comparable return (100.0 * getSizeUsed()) / getMaxSize(); } - /** - * Historical nodes can be 'decommissioned', which instructs Coordinator to move segments from them according to - * the percent of move operations diverted from normal balancer moves for this purpose by - * {@link CoordinatorDynamicConfig#getDecommissioningMaxPercentOfMaxSegmentsToMove()}. The mechanism allows draining - * segments from nodes which are planned for replacement. - * @return true if the node is decommissioning - */ public boolean isDecommissioning() { return isDecommissioning; @@ -108,46 +204,7 @@ public class ServerHolder implements Comparable public long getAvailableSize() { - long maxSize = getMaxSize(); - long sizeUsed = getSizeUsed(); - long availableSize = maxSize - sizeUsed; - - log.debug( - "Server[%s], MaxSize[%,d], CurrSize[%,d], QueueSize[%,d], SizeUsed[%,d], AvailableSize[%,d]", - server.getName(), - maxSize, - getCurrServerSize(), - getLoadQueueSize(), - sizeUsed, - availableSize - ); - - return availableSize; - } - - public boolean isServingSegment(DataSegment segment) - { - return isServingSegment(segment.getId()); - } - - public boolean isLoadingSegment(DataSegment segment) - { - return peon.getSegmentsToLoad().contains(segment); - } - - public boolean isDroppingSegment(DataSegment segment) - { - return peon.getSegmentsToDrop().contains(segment); - } - - public int getNumberOfSegmentsInQueue() - { - return peon.getNumberOfSegmentsInQueue(); - } - - public boolean isServingSegment(SegmentId segmentId) - { - return server.getSegment(segmentId) != null; + return getMaxSize() - getSizeUsed(); } /** @@ -165,31 +222,177 @@ public class ServerHolder implements Comparable public boolean canLoadSegment(DataSegment segment) { return !isDecommissioning - && !isServingSegment(segment.getId()) - && !isLoadingSegment(segment) - && (maxSegmentsInLoadQueue == 0 || maxSegmentsInLoadQueue > peon.getNumberOfSegmentsInQueue()) + && !hasSegmentLoaded(segment.getId()) + && getActionOnSegment(segment) == null + && totalAssignmentsInRun < maxAssignmentsInRun && getAvailableSize() >= segment.getSize(); } + public SegmentAction getActionOnSegment(DataSegment segment) + { + return queuedSegments.get(segment); + } + + /** + * Segments queued for load, drop or move on this server. + *

    + *
  • Contains segments present in the queue when the current coordinator run started.
  • + *
  • Contains segments added to the queue during the current run.
  • + *
  • Maps replicating segments to LOAD rather than REPLICATE for simplicity.
  • + *
  • Does not contain segments whose actions were cancelled.
  • + *
+ */ + public Map getQueuedSegments() + { + return new HashMap<>(queuedSegments); + } + + /** + * Segments that are expected to be loaded on this server once all the + * operations in progress have completed. + */ + public Set getProjectedSegments() + { + return projectedSegments; + } + + /** + * Segments that are currently in the queue for being loaded on this server. + * This does not include segments that are being moved to this server. + */ + public List getLoadingSegments() + { + final List loadingSegments = new ArrayList<>(); + queuedSegments.forEach((segment, action) -> { + if (action == SegmentAction.LOAD) { + loadingSegments.add(segment); + } + }); + + return loadingSegments; + } + + /** + * Segments that are currently loaded on this server. + */ + public Collection getServedSegments() + { + return server.iterateAllSegments(); + } + + /** + * Returns true if this server has the segment loaded and is not dropping it. + */ + public boolean isServingSegment(DataSegment segment) + { + return hasSegmentLoaded(segment.getId()) && getActionOnSegment(segment) == null; + } + + public boolean isLoadingSegment(DataSegment segment) + { + return getActionOnSegment(segment) == SegmentAction.LOAD; + } + + public boolean isDroppingSegment(DataSegment segment) + { + return getActionOnSegment(segment) == SegmentAction.DROP; + } + + public int getNumMovingSegments() + { + return movingSegmentCount; + } + + public int getNumLoadingReplicas() + { + return loadingReplicaCount; + } + + public boolean startOperation(SegmentAction action, DataSegment segment) + { + if (queuedSegments.containsKey(segment)) { + return false; + } + + if (action.isLoad()) { + ++totalAssignmentsInRun; + } + + addToQueuedSegments(segment, simplify(action)); + return true; + } + + public boolean cancelOperation(SegmentAction action, DataSegment segment) + { + // Cancel only if the action is currently in queue + final SegmentAction queuedAction = queuedSegments.get(segment); + if (queuedAction != simplify(action)) { + return false; + } + + // Try cancelling the operation on the peon + // MOVE_FROM operations are not sent to the peon, so they can be considered cancelled + if (queuedAction == SegmentAction.MOVE_FROM || peon.cancelOperation(segment)) { + removeFromQueuedSegments(segment, queuedAction); + return true; + } else { + return false; + } + } + + public boolean hasSegmentLoaded(SegmentId segmentId) + { + return server.getSegment(segmentId) != null; + } + + public boolean isRealtimeServer() + { + return server.getType() == ServerType.REALTIME + || server.getType() == ServerType.INDEXER_EXECUTOR; + } + + private SegmentAction simplify(SegmentAction action) + { + return action == SegmentAction.REPLICATE ? SegmentAction.LOAD : action; + } + + private void addToQueuedSegments(DataSegment segment, SegmentAction action) + { + queuedSegments.put(segment, action); + + // Add to projected if load is started, remove from projected if drop has started + if (action.isLoad()) { + projectedSegments.add(segment); + sizeOfLoadingSegments += segment.getSize(); + } else { + projectedSegments.remove(segment); + if (action == SegmentAction.DROP) { + sizeOfDroppingSegments += segment.getSize(); + } + // MOVE_FROM actions graduate to DROP after the corresponding MOVE_TO has finished + // Do not consider size delta until then, otherwise we might over-assign the server + } + } + + private void removeFromQueuedSegments(DataSegment segment, SegmentAction action) + { + queuedSegments.remove(segment); + + if (action.isLoad()) { + projectedSegments.remove(segment); + sizeOfLoadingSegments -= segment.getSize(); + } else { + projectedSegments.add(segment); + if (action == SegmentAction.DROP) { + sizeOfDroppingSegments -= segment.getSize(); + } + } + } + @Override public int compareTo(ServerHolder serverHolder) { - int result = Long.compare(getAvailableSize(), serverHolder.getAvailableSize()); - if (result != 0) { - return result; - } - - result = server.getHost().compareTo(serverHolder.server.getHost()); - if (result != 0) { - return result; - } - - result = server.getTier().compareTo(serverHolder.server.getTier()); - if (result != 0) { - return result; - } - - return server.getType().compareTo(serverHolder.server.getType()); + return MORE_AVAILABLE_SIZE_SERVER_FIRST.compare(this, serverHolder); } @Override @@ -203,16 +406,9 @@ public class ServerHolder implements Comparable } ServerHolder that = (ServerHolder) o; - - if (!this.server.getHost().equals(that.server.getHost())) { - return false; - } - - if (!this.server.getTier().equals(that.getServer().getTier())) { - return false; - } - - return this.server.getType().equals(that.getServer().getType()); + return Objects.equals(server.getHost(), that.server.getHost()) + && Objects.equals(server.getTier(), that.server.getTier()) + && Objects.equals(server.getType(), that.server.getType()); } @Override @@ -220,4 +416,10 @@ public class ServerHolder implements Comparable { return Objects.hash(server.getHost(), server.getTier(), server.getType()); } + + @Override + public String toString() + { + return "ServerHolder{" + server.getHost() + "}"; + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/BalancerSegmentHolder.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerSegmentHolder.java similarity index 65% rename from server/src/main/java/org/apache/druid/server/coordinator/BalancerSegmentHolder.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerSegmentHolder.java index b842e3f5fb8..8196448064a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/BalancerSegmentHolder.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerSegmentHolder.java @@ -17,33 +17,28 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; -import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; /** + * Represents a segment picked for moving by a balancer strategy. */ public class BalancerSegmentHolder { - private final ImmutableDruidServer fromServer; + private final ServerHolder server; private final DataSegment segment; - // This is a pretty fugly hard coding of the maximum lifetime - private int lifetime = 15; - - public BalancerSegmentHolder( - ImmutableDruidServer fromServer, - DataSegment segment - ) + public BalancerSegmentHolder(ServerHolder server, DataSegment segment) { - this.fromServer = fromServer; + this.server = server; this.segment = segment; } - public ImmutableDruidServer getFromServer() + public ServerHolder getServer() { - return fromServer; + return server; } public DataSegment getSegment() @@ -51,13 +46,4 @@ public class BalancerSegmentHolder return segment; } - public int getLifetime() - { - return lifetime; - } - - public void reduceLifetime() - { - lifetime--; - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java new file mode 100644 index 00000000000..7f27648d3f8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java @@ -0,0 +1,89 @@ +/* + * 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.druid.server.coordinator.balancer; + +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.duty.BalanceSegments; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.timeline.DataSegment; + +import javax.annotation.Nullable; +import java.util.Iterator; +import java.util.List; +import java.util.NavigableSet; + +/** + * This interface describes the coordinator balancing strategy, which is responsible for making decisions on where + * to place {@link DataSegment}s on historical servers (described by {@link ServerHolder}). The balancing strategy + * is used by {@link org.apache.druid.server.coordinator.rules.LoadRule} to assign and drop segments, and by + * {@link BalanceSegments} to migrate segments between historicals. + */ +public interface BalancerStrategy +{ + + /** + * Finds the best server to move a segment to according to the balancing strategy. + * + * @param proposalSegment segment to move + * @param sourceServer Server the segment is currently placed on. + * @param destinationServers servers to consider as move destinations + * @return The server to move to, or null if no move should be made or no server is suitable + */ + @Nullable + ServerHolder findDestinationServerToMoveSegment( + DataSegment proposalSegment, + ServerHolder sourceServer, + List destinationServers + ); + + /** + * Finds the best servers on which to place the {@code proposalSegment}. + * This method can be used both for placing the first copy of a segment + * in the tier or a replica of the segment. + * + * @param proposalSegment segment to place on servers + * @param serverHolders servers to consider as segment homes + * @return Iterator over the best servers (in order) on which the segment + * can be placed. + */ + Iterator findServersToLoadSegment( + DataSegment proposalSegment, + List serverHolders + ); + + /** + * Returns an iterator for a set of servers to drop from, ordered by preference of which server to drop from first + * for a given drop strategy. One or more segments may be dropped, depending on how much the segment is + * over-replicated. + * @param toDropSegment segment to drop from one or more servers + * @param serverHolders set of historicals to consider dropping from + * @return Iterator for set of historicals, ordered by drop preference + */ + Iterator pickServersToDropSegment(DataSegment toDropSegment, NavigableSet serverHolders); + + /** + * Add balancing strategy stats during the 'balanceTier' operation of + * {@link BalanceSegments} to be included + * @param tier historical tier being balanced + * @param stats stats object to add balancing strategy stats to + * @param serverHolderList servers in tier being balanced + */ + void emitStats(String tier, CoordinatorRunStats stats, List serverHolderList); +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategyFactory.java similarity index 74% rename from server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategyFactory.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategyFactory.java index 7e79c9a4455..f27341fd3e0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategyFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; @@ -25,10 +25,10 @@ import com.google.common.util.concurrent.ListeningExecutorService; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "strategy", defaultImpl = CostBalancerStrategyFactory.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "diskNormalized", value = DiskNormalizedCostBalancerStrategyFactory.class), - @JsonSubTypes.Type(name = "cost", value = CostBalancerStrategyFactory.class), - @JsonSubTypes.Type(name = "cachingCost", value = CachingCostBalancerStrategyFactory.class), - @JsonSubTypes.Type(name = "random", value = RandomBalancerStrategyFactory.class), + @JsonSubTypes.Type(name = "cost", value = CostBalancerStrategyFactory.class), + @JsonSubTypes.Type(name = "cachingCost", value = CachingCostBalancerStrategyFactory.class), + @JsonSubTypes.Type(name = "diskNormalized", value = DiskNormalizedCostBalancerStrategyFactory.class), + @JsonSubTypes.Type(name = "random", value = RandomBalancerStrategyFactory.class) }) public interface BalancerStrategyFactory { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategy.java similarity index 82% rename from server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategy.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategy.java index 03f16b1082f..424657991cf 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategy.java @@ -17,12 +17,12 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListeningExecutorService; -import org.apache.druid.server.coordinator.cost.ClusterCostCache; +import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; import java.util.Collections; @@ -43,15 +43,8 @@ public class CachingCostBalancerStrategy extends CostBalancerStrategy @Override protected double computeCost(DataSegment proposalSegment, ServerHolder server, boolean includeCurrentServer) { - final long proposalSegmentSize = proposalSegment.getSize(); - - // (optional) Don't include server if it is already serving segment - if (!includeCurrentServer && server.isServingSegment(proposalSegment)) { - return Double.POSITIVE_INFINITY; - } - - // Don't calculate cost if the server doesn't have enough space or is loading the segment - if (proposalSegmentSize > server.getAvailableSize() || server.isLoadingSegment(proposalSegment)) { + // (optional) Don't include server if it cannot load the segment + if (!includeCurrentServer && !server.canLoadSegment(proposalSegment)) { return Double.POSITIVE_INFINITY; } @@ -63,11 +56,15 @@ public class CachingCostBalancerStrategy extends CostBalancerStrategy cost += costCacheForLoadingSegments(server).computeCost(serverName, proposalSegment); // minus the cost of the segment itself - if (server.isServingSegment(proposalSegment)) { + if (server.isServingSegment(proposalSegment) || server.isLoadingSegment(proposalSegment)) { cost -= costCacheForSegments(server, Collections.singleton(proposalSegment)) .computeCost(serverName, proposalSegment); } + // minus the costs of segments that are being dropped + cost -= costCacheForSegments(server, server.getPeon().getSegmentsToDrop()) + .computeCost(serverName, proposalSegment); + // minus the costs of segments that are marked to be dropped cost -= costCacheForSegments(server, server.getPeon().getSegmentsMarkedToDrop()) .computeCost(serverName, proposalSegment); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyConfig.java similarity index 95% rename from server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyConfig.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyConfig.java index eb0a668301d..b99126d4551 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java similarity index 98% rename from server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java index 1741087e8c5..726dffe6904 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; @@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordinator.cost.ClusterCostCache; import org.apache.druid.timeline.DataSegment; import java.util.concurrent.CancellationException; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/cost/ClusterCostCache.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/ClusterCostCache.java similarity index 98% rename from server/src/main/java/org/apache/druid/server/coordinator/cost/ClusterCostCache.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/ClusterCostCache.java index 84bd317180a..568e7e383af 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/cost/ClusterCostCache.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/ClusterCostCache.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.cost; +package org.apache.druid.server.coordinator.balancer; import com.google.common.base.Preconditions; import org.apache.druid.timeline.DataSegment; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategy.java similarity index 65% rename from server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategy.java index de6d3824f1c..6d165c1f349 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategy.java @@ -17,9 +17,9 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; -import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -27,19 +27,22 @@ import org.apache.commons.math3.util.FastMath; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import java.util.ArrayList; -import java.util.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.NavigableSet; -import java.util.concurrent.ThreadLocalRandom; +import java.util.PriorityQueue; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; public class CostBalancerStrategy implements BalancerStrategy { @@ -52,6 +55,14 @@ public class CostBalancerStrategy implements BalancerStrategy private static final double MILLIS_IN_HOUR = 3_600_000.0; private static final double MILLIS_FACTOR = MILLIS_IN_HOUR / LAMBDA; + /** + * Comparator that prioritizes servers by cost. Cheaper servers come before + * costlier servers. Servers with the same cost may appear in a random order. + */ + private static final Comparator> CHEAPEST_SERVERS_FIRST + = Comparator., Double>comparing(pair -> pair.lhs) + .thenComparing(pair -> pair.rhs); + /** * This defines the unnormalized cost function between two segments. * @@ -187,23 +198,28 @@ public class CostBalancerStrategy implements BalancerStrategy } @Override - public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List serverHolders) + public Iterator findServersToLoadSegment( + DataSegment proposalSegment, + List serverHolders + ) { - ServerHolder holder = chooseBestServer(proposalSegment, serverHolders, false).rhs; - if (holder != null && !holder.isServingSegment(proposalSegment)) { - return holder; - } - return null; + return getServersByPlacementCost(proposalSegment, serverHolders, false, "findServersToLoadSegment"); } @Override - public ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List serverHolders) + public ServerHolder findDestinationServerToMoveSegment( + DataSegment proposalSegment, + ServerHolder sourceServer, + List serverHolders + ) { - return chooseBestServer(proposalSegment, serverHolders, true).rhs; + Iterator servers = + getServersByPlacementCost(proposalSegment, serverHolders, true, "findServerToMoveSegment"); + return servers.hasNext() ? servers.next() : null; } - static double computeJointSegmentsCost(final DataSegment segment, final Iterable segmentSet) + public static double computeJointSegmentsCost(DataSegment segment, Iterable segmentSet) { double totalCost = 0; for (DataSegment s : segmentSet) { @@ -213,34 +229,17 @@ public class CostBalancerStrategy implements BalancerStrategy } @Override - public Iterator pickServersToDrop(DataSegment toDrop, NavigableSet serverHolders) + public Iterator pickServersToDropSegment( + DataSegment segmentToDrop, + NavigableSet serverHolders + ) { - List>> futures = new ArrayList<>(); + List serversByCost = Lists.newArrayList( + getServersByPlacementCost(segmentToDrop, serverHolders, true, "pickServersToDropSegment") + ); - for (final ServerHolder server : serverHolders) { - futures.add( - exec.submit( - () -> Pair.of(computeCost(toDrop, server, true), server) - ) - ); - } - - final ListenableFuture>> resultsFuture = Futures.allAsList(futures); - - try { - // results is an un-ordered list of a pair consisting of the 'cost' of a segment being on a server and the server - List> results = resultsFuture.get(1, TimeUnit.MINUTES); - return results.stream() - // Comparator.comapringDouble will order by lowest cost... - // reverse it because we want to drop from the highest cost servers first - .sorted(Comparator.comparingDouble((Pair o) -> o.lhs).reversed()) - .map(x -> x.rhs).collect(Collectors.toList()) - .iterator(); - } - catch (Exception e) { - alertOnFailure(e, "pick drop server"); - } - return Collections.emptyIterator(); + // Prioritize drop from highest cost servers + return Lists.reverse(serversByCost).iterator(); } /** @@ -281,7 +280,7 @@ public class CostBalancerStrategy implements BalancerStrategy { double cost = 0; for (ServerHolder server : serverHolders) { - for (DataSegment segment : server.getServer().iterateAllSegments()) { + for (DataSegment segment : server.getServedSegments()) { cost += computeJointSegmentsCost(segment, segment); } } @@ -289,15 +288,16 @@ public class CostBalancerStrategy implements BalancerStrategy } @Override - public void emitStats(String tier, CoordinatorStats stats, List serverHolderList) + public void emitStats(String tier, CoordinatorRunStats stats, List serverHolderList) { final double initialTotalCost = calculateInitialTotalCost(serverHolderList); final double normalization = calculateNormalization(serverHolderList); final double normalizedInitialCost = initialTotalCost / normalization; - stats.addToTieredStat("initialCost", tier, (long) initialTotalCost); - stats.addToTieredStat("normalization", tier, (long) normalization); - stats.addToTieredStat("normalizedInitialCostTimesOneThousand", tier, (long) (normalizedInitialCost * 1000)); + final RowKey rowKey = RowKey.forTier(tier); + stats.add(Stats.Balancer.RAW_COST, rowKey, (long) initialTotalCost); + stats.add(Stats.Balancer.NORMALIZATION_COST, rowKey, (long) normalization); + stats.add(Stats.Balancer.NORMALIZED_COST_X_1000, rowKey, (long) (normalizedInitialCost * 1000)); log.info( "[%s]: Initial Total Cost: [%f], Normalization: [%f], Initial Normalized Cost: [%f]", @@ -311,57 +311,42 @@ public class CostBalancerStrategy implements BalancerStrategy final boolean includeCurrentServer ) { - final long proposalSegmentSize = proposalSegment.getSize(); - - // (optional) Don't include server if it is already serving segment - if (!includeCurrentServer && server.isServingSegment(proposalSegment)) { - return Double.POSITIVE_INFINITY; - } - - // Don't calculate cost if the server doesn't have enough space or is loading the segment - if (proposalSegmentSize > server.getAvailableSize() || server.isLoadingSegment(proposalSegment)) { + // (optional) Don't include server if it cannot load the segment + if (!includeCurrentServer && !server.canLoadSegment(proposalSegment)) { return Double.POSITIVE_INFINITY; } // The contribution to the total cost of a given server by proposing to move the segment to that server is... double cost = 0d; - // the sum of the costs of other (exclusive of the proposalSegment) segments on the server - cost += computeJointSegmentsCost( - proposalSegment, - Iterables.filter(server.getServer().iterateAllSegments(), segment -> !proposalSegment.equals(segment)) - ); + // the sum of the costs of segments expected to be on the server (loaded + loading - dropping) + Set projectedSegments = server.getProjectedSegments(); + cost += computeJointSegmentsCost(proposalSegment, projectedSegments); - // plus the costs of segments that will be loaded - cost += computeJointSegmentsCost(proposalSegment, server.getPeon().getSegmentsToLoad()); - - // minus the costs of segments that are marked to be dropped - cost -= computeJointSegmentsCost(proposalSegment, server.getPeon().getSegmentsMarkedToDrop()); + // minus the self cost of the segment + if (projectedSegments.contains(proposalSegment)) { + cost -= computeJointSegmentsCost(proposalSegment, proposalSegment); + } return cost; } /** - * For assignment, we want to move to the lowest cost server that isn't already serving the segment. + * Returns an iterator over the servers, ordered by increasing cost for + * placing the given segment on that server. * - * @param proposalSegment A DataSegment that we are proposing to move. - * @param serverHolders An iterable of ServerHolders for a particular tier. - * - * @return A ServerHolder with the new home for a segment. + * @param includeCurrentServer true if the server already serving a replica + * of this segment should be included in the results */ - - protected Pair chooseBestServer( - final DataSegment proposalSegment, - final Iterable serverHolders, - final boolean includeCurrentServer + private Iterator getServersByPlacementCost( + DataSegment proposalSegment, + Iterable serverHolders, + boolean includeCurrentServer, + String action ) { - final Pair noServer = Pair.of(Double.POSITIVE_INFINITY, null); - Pair bestServer = noServer; - - List>> futures = new ArrayList<>(); - - for (final ServerHolder server : serverHolders) { + final List>> futures = new ArrayList<>(); + for (ServerHolder server : serverHolders) { futures.add( exec.submit( () -> Pair.of(computeCost(proposalSegment, server, includeCurrentServer), server) @@ -369,30 +354,23 @@ public class CostBalancerStrategy implements BalancerStrategy ); } - final ListenableFuture>> resultsFuture = Futures.allAsList(futures); - final List> bestServers = new ArrayList<>(); - bestServers.add(bestServer); + final PriorityQueue> costPrioritizedServers = + new PriorityQueue<>(CHEAPEST_SERVERS_FIRST); try { - for (Pair server : resultsFuture.get(1, TimeUnit.MINUTES)) { - if (server.lhs <= bestServers.get(0).lhs) { - if (server.lhs < bestServers.get(0).lhs) { - bestServers.clear(); - } - bestServers.add(server); - } - } - // If the best server list contains server whose cost of serving the segment is INFINITE then this means - // no usable servers are found so return a null server so that segment assignment does not happen - if (bestServers.get(0).lhs.isInfinite()) { - return noServer; - } - // Randomly choose a server from the best servers - bestServer = bestServers.get(ThreadLocalRandom.current().nextInt(bestServers.size())); + // 1 minute is the typical time for a full run of all historical management duties + // and is more than enough time for the cost computation of a single segment + costPrioritizedServers.addAll( + Futures.allAsList(futures).get(1, TimeUnit.MINUTES) + ); } catch (Exception e) { - alertOnFailure(e, "choose best load server"); + alertOnFailure(e, action); } - return bestServer; + + // Include current server only if specified + return costPrioritizedServers.stream() + .filter(pair -> includeCurrentServer || pair.rhs.canLoadSegment(proposalSegment)) + .map(pair -> pair.rhs).iterator(); } private void alertOnFailure(Exception e, String action) @@ -404,7 +382,6 @@ public class CostBalancerStrategy implements BalancerStrategy } final boolean hasTimedOut = e instanceof TimeoutException; - final String message = StringUtils.format( "Cost balancer strategy %s in action [%s].%s", hasTimedOut ? "timed out" : "failed", action, diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyFactory.java similarity index 95% rename from server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategyFactory.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyFactory.java index c846acf3e8a..3085f35b6b8 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.google.common.util.concurrent.ListeningExecutorService; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategy.java similarity index 94% rename from server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategy.java index 8e559b4a274..cee292930cf 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategy.java @@ -17,9 +17,10 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.google.common.util.concurrent.ListeningExecutorService; +import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; public class DiskNormalizedCostBalancerStrategy extends CostBalancerStrategy diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategyFactory.java similarity index 95% rename from server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyFactory.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategyFactory.java index 10a32bce274..9c404b97e55 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategyFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.google.common.util.concurrent.ListeningExecutorService; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategy.java similarity index 50% rename from server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategy.java index d70d85016c3..98b1d8bdc70 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategy.java @@ -17,8 +17,10 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.timeline.DataSegment; import java.util.ArrayList; @@ -26,34 +28,45 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.NavigableSet; -import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; +/** + * A simple {@link BalancerStrategy} that + *
    + *
  • assigns segments randomly amongst eligible servers
  • + *
  • performs no balancing
  • + *
+ */ public class RandomBalancerStrategy implements BalancerStrategy { @Override - public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List serverHolders) + public Iterator findServersToLoadSegment( + DataSegment proposalSegment, + List serverHolders + ) { - // filter out servers whose avaialable size is less than required for this segment and those already serving this segment - final List usableServerHolders = serverHolders.stream().filter( - serverHolder -> serverHolder.getAvailableSize() >= proposalSegment.getSize() && !serverHolder.isServingSegment( - proposalSegment) - ).collect(Collectors.toList()); - if (usableServerHolders.size() == 0) { - return null; - } else { - return usableServerHolders.get(ThreadLocalRandom.current().nextInt(usableServerHolders.size())); - } + // Filter out servers which cannot load this segment + final List usableServerHolders = + serverHolders.stream() + .filter(server -> server.canLoadSegment(proposalSegment)) + .collect(Collectors.toList()); + Collections.shuffle(usableServerHolders); + return usableServerHolders.iterator(); } @Override - public ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List serverHolders) + public ServerHolder findDestinationServerToMoveSegment( + DataSegment proposalSegment, + ServerHolder sourceServer, + List serverHolders + ) { - return null; //To change body of implemented methods use File | Settings | File Templates. + // This strategy does not do any balancing + return null; } @Override - public Iterator pickServersToDrop(DataSegment toDropSegment, NavigableSet serverHolders) + public Iterator pickServersToDropSegment(DataSegment toDropSegment, NavigableSet serverHolders) { List serverList = new ArrayList<>(serverHolders); Collections.shuffle(serverList); @@ -61,7 +74,7 @@ public class RandomBalancerStrategy implements BalancerStrategy } @Override - public void emitStats(String tier, CoordinatorStats stats, List serverHolderList) + public void emitStats(String tier, CoordinatorRunStats stats, List serverHolderList) { } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategyFactory.java similarity index 95% rename from server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategyFactory.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategyFactory.java index c02554050eb..2655df53380 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategyFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.google.common.util.concurrent.ListeningExecutorService; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSampler.java similarity index 74% rename from server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSampler.java index 751f2597e6b..474347215aa 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSampler.java @@ -17,28 +17,50 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.SegmentAction; import org.apache.druid.timeline.DataSegment; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Function; -final class ReservoirSegmentSampler +public final class ReservoirSegmentSampler { private static final EmittingLogger log = new EmittingLogger(ReservoirSegmentSampler.class); - static List getRandomBalancerSegmentHolders( - final List serverHolders, - Set broadcastDatasources, - int k + /** + * Picks segments from the given set of servers that can be moved to other + * servers for balancing. + * + * @param serverHolders Set of historicals to consider for picking segments + * @param maxSegmentsToPick Maximum number of segments to pick + * @param broadcastDatasources Segments belonging to these datasources will not + * be picked for balancing, since they should be + * loaded on all servers anyway. + * @return Iterator over {@link BalancerSegmentHolder}s, each of which contains + * a segment picked for moving and the server from which it was picked. + */ + public static List pickMovableSegmentsFrom( + List serverHolders, + int maxSegmentsToPick, + Function> segmentProvider, + Set broadcastDatasources ) { - List holders = new ArrayList<>(k); + if (maxSegmentsToPick == 0 || serverHolders.isEmpty()) { + return Collections.emptyList(); + } + + final List pickedSegments = new ArrayList<>(maxSegmentsToPick); int numSoFar = 0; for (ServerHolder server : serverHolders) { @@ -47,25 +69,29 @@ final class ReservoirSegmentSampler continue; } - for (DataSegment segment : server.getServer().iterateAllSegments()) { + final Collection movableSegments = segmentProvider.apply(server); + for (DataSegment segment : movableSegments) { if (broadcastDatasources.contains(segment.getDataSource())) { // we don't need to rebalance segments that were assigned via broadcast rules continue; + } else if (server.getActionOnSegment(segment) == SegmentAction.MOVE_FROM) { + // Do not pick a segment which is already being moved + continue; } - if (numSoFar < k) { - holders.add(new BalancerSegmentHolder(server.getServer(), segment)); + if (numSoFar < maxSegmentsToPick) { + pickedSegments.add(new BalancerSegmentHolder(server, segment)); numSoFar++; continue; } int randNum = ThreadLocalRandom.current().nextInt(numSoFar + 1); - if (randNum < k) { - holders.set(randNum, new BalancerSegmentHolder(server.getServer(), segment)); + if (randNum < maxSegmentsToPick) { + pickedSegments.set(randNum, new BalancerSegmentHolder(server, segment)); } numSoFar++; } } - return holders; + return pickedSegments; } /** @@ -150,7 +176,7 @@ final class ReservoirSegmentSampler } } if (fromServerHolder != null) { - return new BalancerSegmentHolder(fromServerHolder.getServer(), proposalSegment); + return new BalancerSegmentHolder(fromServerHolder, proposalSegment); } else { return null; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/cost/SegmentsCostCache.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/SegmentsCostCache.java similarity index 99% rename from server/src/main/java/org/apache/druid/server/coordinator/cost/SegmentsCostCache.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/SegmentsCostCache.java index 9271de28425..8c67da3748b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/cost/SegmentsCostCache.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/SegmentsCostCache.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.cost; +package org.apache.druid.server.coordinator.balancer; import com.google.common.base.Preconditions; import com.google.common.collect.Ordering; @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.DurationGranularity; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.server.coordinator.CostBalancerStrategy; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/cost/ServerCostCache.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/ServerCostCache.java similarity index 98% rename from server/src/main/java/org/apache/druid/server/coordinator/cost/ServerCostCache.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/ServerCostCache.java index 7897569197b..d0dd61d1d15 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/cost/ServerCostCache.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/ServerCostCache.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.cost; +package org.apache.druid.server.coordinator.balancer; import com.google.common.base.Preconditions; import org.apache.druid.timeline.DataSegment; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java new file mode 100644 index 00000000000..a5ab8d0d477 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java @@ -0,0 +1,237 @@ +/* + * 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.druid.server.coordinator.balancer; + +import com.google.common.collect.Lists; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig; +import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; + +import javax.annotation.Nullable; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Balances segments within the servers of a tier using the balancer strategy. + * Segments are prioritized for move in the following order: + *
    + *
  • Segments loaded on decommissioning servers
  • + *
  • Segments loading on active servers
  • + *
  • Segments loaded on active servers
  • + *
+ */ +public class TierSegmentBalancer +{ + private static final EmittingLogger log = new EmittingLogger(TierSegmentBalancer.class); + + private final String tier; + private final DruidCoordinatorRuntimeParams params; + private final StrategicSegmentAssigner segmentAssigner; + + private final BalancerStrategy strategy; + private final SegmentLoadingConfig loadingConfig; + private final CoordinatorRunStats runStats; + + private final Set allServers; + private final List activeServers; + private final List decommissioningServers; + private final int totalMaxSegmentsToMove; + + private final int movingSegmentCount; + + public TierSegmentBalancer( + String tier, + Set servers, + DruidCoordinatorRuntimeParams params + ) + { + this.tier = tier; + this.params = params; + this.segmentAssigner = params.getSegmentAssigner(); + + this.strategy = params.getBalancerStrategy(); + this.loadingConfig = params.getSegmentLoadingConfig(); + this.totalMaxSegmentsToMove = loadingConfig.getMaxSegmentsToMove(); + this.runStats = segmentAssigner.getStats(); + + Map> partitions = + servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning)); + this.decommissioningServers = partitions.get(true); + this.activeServers = partitions.get(false); + this.allServers = servers; + + this.movingSegmentCount = activeServers.stream().mapToInt(ServerHolder::getNumMovingSegments).sum(); + } + + public void run() + { + if (activeServers.isEmpty() || (activeServers.size() <= 1 && decommissioningServers.isEmpty())) { + log.warn( + "Skipping balance for tier [%s] with [%d] active servers and [%d] decomissioning servers.", + tier, activeServers.size(), decommissioningServers.size() + ); + return; + } + + log.info( + "Moving max [%d] segments in tier [%s] with [%d] active servers and" + + " [%d] decommissioning servers. There are [%d] segments already in queue.", + totalMaxSegmentsToMove, tier, activeServers.size(), decommissioningServers.size(), movingSegmentCount + ); + + // Move segments from decommissioning to active servers + int movedDecommSegments = 0; + if (!decommissioningServers.isEmpty()) { + int maxDecommPercentToMove = loadingConfig.getPercentDecommSegmentsToMove(); + int maxDecommSegmentsToMove = (int) Math.ceil(totalMaxSegmentsToMove * (maxDecommPercentToMove / 100.0)); + movedDecommSegments += + moveSegmentsFromTo(decommissioningServers, activeServers, maxDecommSegmentsToMove); + log.info( + "Moved [%d] segments out of max [%d (%d%%)] from decommissioning to active servers in tier [%s].", + movedDecommSegments, maxDecommSegmentsToMove, maxDecommPercentToMove, tier + ); + } + + // Move segments across active servers + int maxGeneralSegmentsToMove = totalMaxSegmentsToMove - movedDecommSegments; + int movedGeneralSegments = + moveSegmentsFromTo(activeServers, activeServers, maxGeneralSegmentsToMove); + log.info( + "Moved [%d] segments out of max [%d] between active servers in tier [%s].", + movedGeneralSegments, maxGeneralSegmentsToMove, tier + ); + + if (loadingConfig.isEmitBalancingStats()) { + strategy.emitStats(tier, runStats, Lists.newArrayList(allServers)); + } + } + + private int moveSegmentsFromTo( + List sourceServers, + List destServers, + int maxSegmentsToMove + ) + { + if (maxSegmentsToMove <= 0 || sourceServers.isEmpty() || destServers.isEmpty()) { + return 0; + } + + final Set broadcastDatasources = params.getBroadcastDatasources(); + + // Always move loading segments first as it is a cheaper operation + List pickedSegments = ReservoirSegmentSampler.pickMovableSegmentsFrom( + sourceServers, + maxSegmentsToMove, + ServerHolder::getLoadingSegments, + broadcastDatasources + ); + int movedCount = moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove); + + // Move loaded segments only if tier is not already busy moving segments + if (movingSegmentCount <= 0) { + maxSegmentsToMove -= movedCount; + pickedSegments = ReservoirSegmentSampler.pickMovableSegmentsFrom( + sourceServers, + maxSegmentsToMove, + server -> server.getServer().iterateAllSegments(), + broadcastDatasources + ); + movedCount += moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove); + } + + return movedCount; + } + + private int moveSegmentsTo( + List destinationServers, + List movableSegments, + int maxSegmentsToMove + ) + { + int processed = 0; + int movedCount = 0; + + final Iterator segmentIterator = movableSegments.iterator(); + while (segmentIterator.hasNext() && processed < maxSegmentsToMove) { + ++processed; + + final BalancerSegmentHolder segmentHolder = segmentIterator.next(); + DataSegment segmentToMove = getLoadableSegment(segmentHolder.getSegment()); + if (segmentToMove != null && + segmentAssigner.moveSegment(segmentToMove, segmentHolder.getServer(), destinationServers)) { + ++movedCount; + } + } + return movedCount; + } + + /** + * Returns a DataSegment with the correct value of loadSpec (as obtained from + * metadata store). This method may return null if there is no snapshot available + * for the underlying datasource or if the segment is unused. + */ + @Nullable + private DataSegment getLoadableSegment(DataSegment segmentToMove) + { + if (!params.getUsedSegments().contains(segmentToMove)) { + markUnmoved("Segment is unused", segmentToMove); + return null; + } + + ImmutableDruidDataSource datasource = params.getDataSourcesSnapshot() + .getDataSource(segmentToMove.getDataSource()); + if (datasource == null) { + markUnmoved("Invalid datasource", segmentToMove); + return null; + } + + DataSegment loadableSegment = datasource.getSegment(segmentToMove.getId()); + if (loadableSegment == null) { + markUnmoved("Invalid segment ID", segmentToMove); + return null; + } + + return loadableSegment; + } + + private void markUnmoved(String reason, DataSegment segment) + { + final RowKey key + = RowKey.builder() + .add(Dimension.TIER, tier) + .add(Dimension.DATASOURCE, segment.getDataSource()) + .add(Dimension.DESCRIPTION, reason) + .build(); + + runStats.add(Stats.Segments.MOVE_SKIPPED, key, 1); + } + +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/cost/package-info.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/package-info.java similarity index 94% rename from server/src/main/java/org/apache/druid/server/coordinator/cost/package-info.java rename to server/src/main/java/org/apache/druid/server/coordinator/balancer/package-info.java index 16281bbebe6..2036dcf1efe 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/cost/package-info.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/package-info.java @@ -18,6 +18,6 @@ */ @ParametersAreNonnullByDefault -package org.apache.druid.server.coordinator.cost; +package org.apache.druid.server.coordinator.balancer; import javax.annotation.ParametersAreNonnullByDefault; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java index 41871dd039d..865fde2a0bc 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java @@ -19,293 +19,45 @@ package org.apache.druid.server.coordinator.duty; -import com.google.common.collect.Lists; -import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.coordinator.BalancerSegmentHolder; -import org.apache.druid.server.coordinator.BalancerStrategy; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.LoadPeonCallback; -import org.apache.druid.server.coordinator.LoadQueuePeon; -import org.apache.druid.server.coordinator.ServerHolder; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.SortedSet; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.stream.Collectors; +import org.apache.druid.server.coordinator.balancer.TierSegmentBalancer; +import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig; /** + * */ public class BalanceSegments implements CoordinatorDuty { - protected static final EmittingLogger log = new EmittingLogger(BalanceSegments.class); - - protected final DruidCoordinator coordinator; - - protected final Map> currentlyMovingSegments = - new HashMap<>(); - - public BalanceSegments(DruidCoordinator coordinator) - { - this.coordinator = coordinator; - } - - protected void reduceLifetimes(String tier) - { - for (BalancerSegmentHolder holder : currentlyMovingSegments.get(tier).values()) { - holder.reduceLifetime(); - if (holder.getLifetime() <= 0) { - log.makeAlert("[%s]: Balancer move segments queue has a segment stuck", tier) - .addData("segment", holder.getSegment().getId()) - .addData("server", holder.getFromServer().getMetadata()) - .emit(); - } - } - } + private static final EmittingLogger log = new EmittingLogger(BalanceSegments.class); @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - final CoordinatorStats stats = new CoordinatorStats(); - params.getDruidCluster().getHistoricals().forEach((String tier, NavigableSet servers) -> { - balanceTier(params, tier, servers, stats); - }); - return params.buildFromExisting().withCoordinatorStats(stats).build(); - } - - private void balanceTier( - DruidCoordinatorRuntimeParams params, - String tier, - SortedSet servers, - CoordinatorStats stats - ) - { - - log.info("Balancing segments in tier [%s]", tier); - if (params.getUsedSegments().size() == 0) { - log.info("Metadata segments are not available. Cannot balance."); - // suppress emit zero stats - return; - } - currentlyMovingSegments.computeIfAbsent(tier, t -> new ConcurrentHashMap<>()); - - if (!currentlyMovingSegments.get(tier).isEmpty()) { - reduceLifetimes(tier); - log.info( - "[%s]: Still waiting on %,d segments to be moved. Skipping balance.", - tier, - currentlyMovingSegments.get(tier).size() - ); - // suppress emit zero stats - return; + if (params.getUsedSegments().isEmpty()) { + log.info("Skipping balance as there are no used segments."); + return params; } - /* - Take as many segments from decommissioning servers as decommissioningMaxPercentOfMaxSegmentsToMove allows and find - the best location for them on active servers. After that, balance segments within active servers pool. - */ - Map> partitions = - servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning)); - final List decommissioningServers = partitions.get(true); - final List activeServers = partitions.get(false); - log.info( - "Found %d active servers, %d decommissioning servers", - activeServers.size(), - decommissioningServers.size() - ); - - if ((decommissioningServers.isEmpty() && activeServers.size() <= 1) || activeServers.isEmpty()) { - log.warn("[%s]: insufficient active servers. Cannot balance.", tier); - // suppress emit zero stats - return; - } - - int numSegments = 0; - for (ServerHolder sourceHolder : servers) { - numSegments += sourceHolder.getServer().getNumSegments(); - } - - if (numSegments == 0) { - log.info("No segments found. Cannot balance."); - // suppress emit zero stats - return; - } - - final int maxSegmentsToMove = Math.min(params.getCoordinatorDynamicConfig().getMaxSegmentsToMove(), numSegments); - - // Prioritize moving segments from decomissioning servers. - int decommissioningMaxPercentOfMaxSegmentsToMove = - params.getCoordinatorDynamicConfig().getDecommissioningMaxPercentOfMaxSegmentsToMove(); - int maxSegmentsToMoveFromDecommissioningNodes = - (int) Math.ceil(maxSegmentsToMove * (decommissioningMaxPercentOfMaxSegmentsToMove / 100.0)); - log.info( - "Processing %d segments for moving from decommissioning servers", - maxSegmentsToMoveFromDecommissioningNodes - ); - Pair decommissioningResult = - balanceServers(params, decommissioningServers, activeServers, maxSegmentsToMoveFromDecommissioningNodes); - - // After moving segments from decomissioning servers, move the remaining segments from the rest of the servers. - int maxGeneralSegmentsToMove = maxSegmentsToMove - decommissioningResult.lhs; - log.info("Processing %d segments for balancing between active servers", maxGeneralSegmentsToMove); - Pair generalResult = - balanceServers(params, activeServers, activeServers, maxGeneralSegmentsToMove); - - int moved = generalResult.lhs + decommissioningResult.lhs; - int unmoved = generalResult.rhs + decommissioningResult.rhs; - if (unmoved == maxSegmentsToMove) { - // Cluster should be alive and constantly adjusting - log.info("No good moves found in tier [%s]", tier); - } - stats.addToTieredStat("unmovedCount", tier, unmoved); - stats.addToTieredStat("movedCount", tier, moved); - - if (params.getCoordinatorDynamicConfig().emitBalancingStats()) { - final BalancerStrategy strategy = params.getBalancerStrategy(); - strategy.emitStats(tier, stats, Lists.newArrayList(servers)); - } - log.info("[%s]: Segments Moved: [%d] Segments Let Alone: [%d]", tier, moved, unmoved); - } - - private Pair balanceServers( - DruidCoordinatorRuntimeParams params, - List toMoveFrom, - List toMoveTo, - int maxSegmentsToMove - ) - { + final DruidCluster cluster = params.getDruidCluster(); + final SegmentLoadingConfig loadingConfig = params.getSegmentLoadingConfig(); + final int maxSegmentsToMove = loadingConfig.getMaxSegmentsToMove(); if (maxSegmentsToMove <= 0) { - log.debug("maxSegmentsToMove is 0; no balancing work can be performed."); - return new Pair<>(0, 0); - } else if (toMoveFrom.isEmpty()) { - log.debug("toMoveFrom is empty; no balancing work can be performed."); - return new Pair<>(0, 0); - } else if (toMoveTo.isEmpty()) { - log.debug("toMoveTo is empty; no balancing work can be peformed."); - return new Pair<>(0, 0); - } - - final BalancerStrategy strategy = params.getBalancerStrategy(); - final int maxIterations = 2 * maxSegmentsToMove; - final int maxToLoad = params.getCoordinatorDynamicConfig().getMaxSegmentsInNodeLoadingQueue(); - int moved = 0, unmoved = 0; - - Iterator segmentsToMove; - // The pick method depends on if the operator has enabled batched segment sampling in the Coorinator dynamic config. - if (params.getCoordinatorDynamicConfig().useBatchedSegmentSampler()) { - segmentsToMove = strategy.pickSegmentsToMove( - toMoveFrom, - params.getBroadcastDatasources(), - maxSegmentsToMove - ); + log.info("Skipping balance as maxSegmentsToMove is [%d].", maxSegmentsToMove); + return params; } else { - segmentsToMove = strategy.pickSegmentsToMove( - toMoveFrom, - params.getBroadcastDatasources(), - params.getCoordinatorDynamicConfig().getPercentOfSegmentsToConsiderPerMove() + log.info( + "Balancing segments in tiers [%s] with maxSegmentsToMove=[%d], maxLifetime=[%d].", + cluster.getTierNames(), maxSegmentsToMove, loadingConfig.getMaxLifetimeInLoadQueue() ); } - //noinspection ForLoopThatDoesntUseLoopVariable - for (int iter = 0; (moved + unmoved) < maxSegmentsToMove; ++iter) { - if (!segmentsToMove.hasNext()) { - log.info("All servers to move segments from are empty, ending run."); - break; - } - final BalancerSegmentHolder segmentToMoveHolder = segmentsToMove.next(); + cluster.getHistoricals().forEach( + (tier, servers) -> new TierSegmentBalancer(tier, servers, params).run() + ); - // DruidCoordinatorRuntimeParams.getUsedSegments originate from SegmentsMetadataManager, i. e. that's a set of segments - // that *should* be loaded. segmentToMoveHolder.getSegment originates from ServerInventoryView, i. e. that may be - // any segment that happens to be loaded on some server, even if it is not used. (Coordinator closes such - // discrepancies eventually via UnloadUnusedSegments). Therefore the picked segmentToMoveHolder's segment may not - // need to be balanced. - boolean needToBalancePickedSegment = params.getUsedSegments().contains(segmentToMoveHolder.getSegment()); - if (needToBalancePickedSegment) { - final DataSegment segmentToMove = segmentToMoveHolder.getSegment(); - final ImmutableDruidServer fromServer = segmentToMoveHolder.getFromServer(); - // we want to leave the server the segment is currently on in the list... - // but filter out replicas that are already serving the segment, and servers with a full load queue - final List toMoveToWithLoadQueueCapacityAndNotServingSegment = - toMoveTo.stream() - .filter(s -> s.getServer().equals(fromServer) || - (!s.isServingSegment(segmentToMove) && - (maxToLoad <= 0 || s.getNumberOfSegmentsInQueue() < maxToLoad))) - .collect(Collectors.toList()); - - if (toMoveToWithLoadQueueCapacityAndNotServingSegment.size() > 0) { - final ServerHolder destinationHolder = - strategy.findNewSegmentHomeBalancer(segmentToMove, toMoveToWithLoadQueueCapacityAndNotServingSegment); - - if (destinationHolder != null && !destinationHolder.getServer().equals(fromServer)) { - if (moveSegment(segmentToMoveHolder, destinationHolder.getServer(), params)) { - moved++; - } else { - unmoved++; - } - } else { - log.debug("Segment [%s] is 'optimally' placed.", segmentToMove.getId()); - unmoved++; - } - } else { - log.debug("No valid movement destinations for segment [%s].", segmentToMove.getId()); - unmoved++; - } - } - if (iter >= maxIterations) { - log.info( - "Unable to select %d remaining candidate segments out of %d total to balance " - + "after %d iterations, ending run.", - (maxSegmentsToMove - moved - unmoved), - maxSegmentsToMove, - iter - ); - break; - } - } - return new Pair<>(moved, unmoved); + return params; } - protected boolean moveSegment( - final BalancerSegmentHolder segment, - final ImmutableDruidServer toServer, - final DruidCoordinatorRuntimeParams params - ) - { - final LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServer.getName()); - - final ImmutableDruidServer fromServer = segment.getFromServer(); - final DataSegment segmentToMove = segment.getSegment(); - final SegmentId segmentId = segmentToMove.getId(); - - if (!toPeon.getSegmentsToLoad().contains(segmentToMove) && - (toServer.getSegment(segmentId) == null) && - new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) { - log.debug("Moving [%s] from [%s] to [%s]", segmentId, fromServer.getName(), toServer.getName()); - - ConcurrentMap movingSegments = - currentlyMovingSegments.get(toServer.getTier()); - movingSegments.put(segmentId, segment); - final LoadPeonCallback callback = moveSuccess -> movingSegments.remove(segmentId); - try { - coordinator - .moveSegment(params, fromServer, toServer, segmentToMove, callback); - return true; - } - catch (Exception e) { - log.makeAlert(e, "[%s] : Moving exception", segmentId).emit(); - callback.execute(false); - } - } - return false; - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java new file mode 100644 index 00000000000..b5825cb084f --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java @@ -0,0 +1,154 @@ +/* + * 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.druid.server.coordinator.duty; + +import com.google.common.util.concurrent.AtomicDouble; +import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Collects stats pertaining to segment availability on different servers. + */ +public class CollectSegmentAndServerStats implements CoordinatorDuty +{ + private static final Logger log = new Logger(CollectSegmentAndServerStats.class); + + private final DruidCoordinator coordinator; + + public CollectSegmentAndServerStats(DruidCoordinator coordinator) + { + this.coordinator = coordinator; + } + + @Override + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) + { + params.getDruidCluster().getHistoricals() + .forEach(this::logHistoricalTierStats); + collectSegmentStats(params); + + StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); + segmentAssigner.makeAlerts(); + + return params; + } + + private void collectSegmentStats(DruidCoordinatorRuntimeParams params) + { + final CoordinatorRunStats stats = params.getCoordinatorStats(); + + final DruidCluster cluster = params.getDruidCluster(); + cluster.getHistoricals().forEach((tier, historicals) -> { + final RowKey rowKey = RowKey.forTier(tier); + stats.add(Stats.Tier.HISTORICAL_COUNT, rowKey, historicals.size()); + long totalCapacity = historicals.stream().map(ServerHolder::getMaxSize).reduce(0L, Long::sum); + stats.add(Stats.Tier.TOTAL_CAPACITY, rowKey, totalCapacity); + }); + + // Collect load queue stats + coordinator.getLoadManagementPeons().forEach((serverName, queuePeon) -> { + final RowKey rowKey = RowKey.builder().add(Dimension.SERVER, serverName).build(); + stats.add(Stats.SegmentQueue.BYTES_TO_LOAD, rowKey, queuePeon.getSizeOfSegmentsToLoad()); + stats.add(Stats.SegmentQueue.NUM_TO_LOAD, rowKey, queuePeon.getSegmentsToLoad().size()); + stats.add(Stats.SegmentQueue.NUM_TO_DROP, rowKey, queuePeon.getSegmentsToDrop().size()); + + queuePeon.getAndResetStats().forEachStat( + (dimValues, stat, statValue) -> + stats.add(stat, createRowKeyForServer(serverName, dimValues), statValue) + ); + }); + + coordinator.getDatasourceToUnavailableSegmentCount().forEach( + (dataSource, numUnavailable) -> + stats.addToDatasourceStat(Stats.Segments.UNAVAILABLE, dataSource, numUnavailable) + ); + + coordinator.getTierToDatasourceToUnderReplicatedCount(false).forEach( + (tier, countsPerDatasource) -> countsPerDatasource.forEach( + (dataSource, underReplicatedCount) -> + stats.addToSegmentStat(Stats.Segments.UNDER_REPLICATED, tier, dataSource, underReplicatedCount) + ) + ); + + // Collect total segment stats + params.getUsedSegmentsTimelinesPerDataSource().forEach( + (dataSource, timeline) -> { + long totalSizeOfUsedSegments = timeline.iterateAllObjects().stream() + .mapToLong(DataSegment::getSize).sum(); + stats.addToDatasourceStat(Stats.Segments.USED_BYTES, dataSource, totalSizeOfUsedSegments); + stats.addToDatasourceStat(Stats.Segments.USED, dataSource, timeline.getNumObjects()); + } + ); + } + + private RowKey createRowKeyForServer(String serverName, Map dimensionValues) + { + final RowKey.Builder builder = RowKey.builder(); + dimensionValues.forEach(builder::add); + builder.add(Dimension.SERVER, serverName); + return builder.build(); + } + + private void logHistoricalTierStats(String tier, Set historicals) + { + final AtomicInteger servedCount = new AtomicInteger(); + final AtomicInteger loadingCount = new AtomicInteger(); + final AtomicInteger droppingCount = new AtomicInteger(); + + final AtomicDouble usageSum = new AtomicDouble(); + final AtomicLong currentBytesSum = new AtomicLong(); + + historicals.forEach(serverHolder -> { + final ImmutableDruidServer server = serverHolder.getServer(); + servedCount.addAndGet(server.getNumSegments()); + currentBytesSum.addAndGet(server.getCurrSize()); + usageSum.addAndGet(100.0f * server.getCurrSize() / server.getMaxSize()); + + final LoadQueuePeon queuePeon = serverHolder.getPeon(); + loadingCount.addAndGet(queuePeon.getSegmentsToLoad().size()); + droppingCount.addAndGet(queuePeon.getSegmentsToDrop().size()); + }); + + final int numHistoricals = historicals.size(); + log.info( + "Tier [%s] is serving [%,d], loading [%,d] and dropping [%,d] segments" + + " across [%d] historicals with average usage [%d GBs], [%.1f%%].", + tier, servedCount.get(), loadingCount.get(), droppingCount.get(), numHistoricals, + (currentBytesSum.get() >> 30) / numHistoricals, usageSum.get() / numHistoricals + ); + } + +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 63267c81b8d..d767c51a925 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -39,16 +39,19 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CompactionStatistics; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; -import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -58,25 +61,9 @@ import java.util.stream.Collectors; public class CompactSegments implements CoordinatorCustomDuty { - static final String COMPACTION_TASK_COUNT = "compactTaskCount"; - static final String AVAILABLE_COMPACTION_TASK_SLOT = "availableCompactionTaskSlot"; - static final String MAX_COMPACTION_TASK_SLOT = "maxCompactionTaskSlot"; - - static final String TOTAL_SIZE_OF_SEGMENTS_SKIPPED = "segmentSizeSkippedCompact"; - static final String TOTAL_COUNT_OF_SEGMENTS_SKIPPED = "segmentCountSkippedCompact"; - static final String TOTAL_INTERVAL_OF_SEGMENTS_SKIPPED = "segmentIntervalSkippedCompact"; - - static final String TOTAL_SIZE_OF_SEGMENTS_AWAITING = "segmentSizeWaitCompact"; - static final String TOTAL_COUNT_OF_SEGMENTS_AWAITING = "segmentCountWaitCompact"; - static final String TOTAL_INTERVAL_OF_SEGMENTS_AWAITING = "segmentIntervalWaitCompact"; - - static final String TOTAL_SIZE_OF_SEGMENTS_COMPACTED = "segmentSizeCompacted"; - static final String TOTAL_COUNT_OF_SEGMENTS_COMPACTED = "segmentCountCompacted"; - static final String TOTAL_INTERVAL_OF_SEGMENTS_COMPACTED = "segmentIntervalCompacted"; - - /** Must be synced with org.apache.druid.indexing.common.task.CompactionTask.TYPE. */ + /** Must be the same as org.apache.druid.indexing.common.task.CompactionTask.TYPE. */ public static final String COMPACTION_TASK_TYPE = "compact"; - /** Must be synced with org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY */ + /** Must be the same as org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY */ public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState"; private static final Logger LOG = new Logger(CompactSegments.class); @@ -100,7 +87,7 @@ public class CompactSegments implements CoordinatorCustomDuty this.policy = policy; this.indexingServiceClient = indexingServiceClient; this.skipLockedIntervals = config.getCompactionSkipLockedIntervals(); - autoCompactionSnapshotPerDataSource.set(new HashMap<>()); + resetCompactionSnapshot(); LOG.info("Scheduling compaction with skipLockedIntervals [%s]", skipLockedIntervals); } @@ -111,144 +98,134 @@ public class CompactSegments implements CoordinatorCustomDuty return skipLockedIntervals; } - @VisibleForTesting - IndexingServiceClient getIndexingServiceClient() - { - return indexingServiceClient; - } - @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - LOG.info("Compact segments"); + LOG.info("Running CompactSegments duty"); final CoordinatorCompactionConfig dynamicConfig = params.getCoordinatorCompactionConfig(); - final CoordinatorStats stats = new CoordinatorStats(); - List compactionConfigList = dynamicConfig.getCompactionConfigs(); - if (dynamicConfig.getMaxCompactionTaskSlots() > 0) { - Map dataSources = - params.getUsedSegmentsTimelinesPerDataSource(); - if (compactionConfigList != null && !compactionConfigList.isEmpty()) { - Map compactionConfigs = compactionConfigList - .stream() - .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); - final List compactionTasks = filterNonCompactionTasks(indexingServiceClient.getActiveTasks()); - - // dataSource -> list of intervals for which compaction will be skipped in this run - final Map> intervalsToSkipCompaction = new HashMap<>(); - - int numEstimatedNonCompleteCompactionTasks = 0; - for (TaskStatusPlus status : compactionTasks) { - final TaskPayloadResponse response = indexingServiceClient.getTaskPayload(status.getId()); - if (response == null) { - throw new ISE("Got a null paylord from overlord for task[%s]", status.getId()); - } - if (COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) { - final ClientCompactionTaskQuery compactionTaskQuery = (ClientCompactionTaskQuery) response.getPayload(); - DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(status.getDataSource()); - if (dataSourceCompactionConfig != null && dataSourceCompactionConfig.getGranularitySpec() != null) { - Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec().getSegmentGranularity(); - if (configuredSegmentGranularity != null - && compactionTaskQuery.getGranularitySpec() != null - && !configuredSegmentGranularity.equals(compactionTaskQuery.getGranularitySpec().getSegmentGranularity())) { - // We will cancel active compaction task if segmentGranularity changes and we will need to - // re-compact the interval - LOG.info("Canceled task[%s] as task segmentGranularity is [%s] but compaction config " - + "segmentGranularity is [%s]", - status.getId(), - compactionTaskQuery.getGranularitySpec().getSegmentGranularity(), - configuredSegmentGranularity); - indexingServiceClient.cancelTask(status.getId()); - continue; - } - } - // Skip interval as the current active compaction task is good - final Interval interval = compactionTaskQuery.getIoConfig().getInputSpec().getInterval(); - intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>()).add(interval); - // Since we keep the current active compaction task running, we count the active task slots - numEstimatedNonCompleteCompactionTasks += findMaxNumTaskSlotsUsedByOneCompactionTask( - compactionTaskQuery.getTuningConfig() - ); - } else { - throw new ISE("task[%s] is not a compactionTask", status.getId()); - } - } - - // Skip all the intervals locked by higher priority tasks for each datasource - // This must be done after the invalid compaction tasks are cancelled - // in the loop above so that their intervals are not considered locked - getLockedIntervalsToSkip(compactionConfigList).forEach( - (dataSource, intervals) -> - intervalsToSkipCompaction - .computeIfAbsent(dataSource, ds -> new ArrayList<>()) - .addAll(intervals) - ); - - final CompactionSegmentIterator iterator = - policy.reset(compactionConfigs, dataSources, intervalsToSkipCompaction); - - int totalCapacity; - if (dynamicConfig.isUseAutoScaleSlots()) { - try { - totalCapacity = indexingServiceClient.getTotalWorkerCapacityWithAutoScale(); - } - catch (Exception e) { - LOG.warn("Failed to get total worker capacity with auto scale slots. Falling back to current capacity count"); - totalCapacity = indexingServiceClient.getTotalWorkerCapacity(); - } - } else { - totalCapacity = indexingServiceClient.getTotalWorkerCapacity(); - } - - final int compactionTaskCapacity = (int) Math.min( - totalCapacity * dynamicConfig.getCompactionTaskSlotRatio(), - dynamicConfig.getMaxCompactionTaskSlots() - ); - final int numAvailableCompactionTaskSlots; - if (numEstimatedNonCompleteCompactionTasks > 0) { - numAvailableCompactionTaskSlots = Math.max( - 0, - compactionTaskCapacity - numEstimatedNonCompleteCompactionTasks - ); - } else { - // compactionTaskCapacity might be 0 if totalWorkerCapacity is low. - // This guarantees that at least one slot is available if - // compaction is enabled and numEstimatedNonCompleteCompactionTasks is 0. - numAvailableCompactionTaskSlots = Math.max(1, compactionTaskCapacity); - } - - LOG.info( - "Found [%d] available task slots for compaction out of [%d] max compaction task capacity", - numAvailableCompactionTaskSlots, - compactionTaskCapacity - ); - stats.addToGlobalStat(AVAILABLE_COMPACTION_TASK_SLOT, numAvailableCompactionTaskSlots); - stats.addToGlobalStat(MAX_COMPACTION_TASK_SLOT, compactionTaskCapacity); - final Map currentRunAutoCompactionSnapshotBuilders = new HashMap<>(); - if (numAvailableCompactionTaskSlots > 0) { - stats.accumulate( - doRun( - compactionConfigs, - currentRunAutoCompactionSnapshotBuilders, - numAvailableCompactionTaskSlots, - iterator - ) - ); - } else { - stats.accumulate(makeStats(currentRunAutoCompactionSnapshotBuilders, 0, iterator)); - } - } else { - LOG.info("compactionConfig is empty. Skip."); - autoCompactionSnapshotPerDataSource.set(new HashMap<>()); - } - } else { - LOG.info("maxCompactionTaskSlots was set to 0. Skip compaction"); - autoCompactionSnapshotPerDataSource.set(new HashMap<>()); + final int maxCompactionTaskSlots = dynamicConfig.getMaxCompactionTaskSlots(); + if (maxCompactionTaskSlots <= 0) { + LOG.info("Skipping compaction as maxCompactionTaskSlots is [%d].", maxCompactionTaskSlots); + resetCompactionSnapshot(); + return params; } - return params.buildFromExisting() - .withCoordinatorStats(stats) - .build(); + List compactionConfigList = dynamicConfig.getCompactionConfigs(); + if (compactionConfigList == null || compactionConfigList.isEmpty()) { + LOG.info("Skipping compaction as compaction config list is empty."); + resetCompactionSnapshot(); + return params; + } + + Map compactionConfigs = compactionConfigList + .stream() + .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); + + // Map from dataSource to list of intervals for which compaction will be skipped in this run + final Map> intervalsToSkipCompaction = new HashMap<>(); + + // Fetch currently running compaction tasks + int busyCompactionTaskSlots = 0; + final List compactionTasks = filterNonCompactionTasks(indexingServiceClient.getActiveTasks()); + for (TaskStatusPlus status : compactionTasks) { + final TaskPayloadResponse response = indexingServiceClient.getTaskPayload(status.getId()); + if (response == null) { + throw new ISE("Could not find payload for active compaction task[%s]", status.getId()); + } else if (!COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) { + throw new ISE( + "Payload of active compaction task[%s] is of invalid type[%s]", + status.getId(), response.getPayload().getType() + ); + } + + final ClientCompactionTaskQuery compactionTaskQuery = (ClientCompactionTaskQuery) response.getPayload(); + DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(status.getDataSource()); + if (cancelTaskIfGranularityChanged(compactionTaskQuery, dataSourceCompactionConfig)) { + continue; + } + + // Skip this interval as the current active compaction task is good + final Interval interval = compactionTaskQuery.getIoConfig().getInputSpec().getInterval(); + intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>()) + .add(interval); + + busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneCompactionTask( + compactionTaskQuery.getTuningConfig() + ); + } + + // Skip all the intervals locked by higher priority tasks for each datasource + // This must be done after the invalid compaction tasks are cancelled + // in the loop above so that their intervals are not considered locked + getLockedIntervalsToSkip(compactionConfigList).forEach( + (dataSource, intervals) -> + intervalsToSkipCompaction + .computeIfAbsent(dataSource, ds -> new ArrayList<>()) + .addAll(intervals) + ); + + // Get iterator over segments to compact and submit compaction tasks + Map dataSources = params.getUsedSegmentsTimelinesPerDataSource(); + final CompactionSegmentIterator iterator = + policy.reset(compactionConfigs, dataSources, intervalsToSkipCompaction); + + final int compactionTaskCapacity = getCompactionTaskCapacity(dynamicConfig); + final int availableCompactionTaskSlots + = getAvailableCompactionTaskSlots(compactionTaskCapacity, busyCompactionTaskSlots); + + final Map currentRunAutoCompactionSnapshotBuilders = new HashMap<>(); + final int numSubmittedCompactionTasks = submitCompactionTasks( + compactionConfigs, + currentRunAutoCompactionSnapshotBuilders, + availableCompactionTaskSlots, + iterator + ); + + final CoordinatorRunStats stats = params.getCoordinatorStats(); + stats.add(Stats.Compaction.MAX_SLOTS, compactionTaskCapacity); + stats.add(Stats.Compaction.AVAILABLE_SLOTS, availableCompactionTaskSlots); + stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks); + addCompactionSnapshotStats(currentRunAutoCompactionSnapshotBuilders, iterator, stats); + + return params; + } + + private void resetCompactionSnapshot() + { + autoCompactionSnapshotPerDataSource.set(Collections.emptyMap()); + } + + /** + * Cancels a currently running compaction task if the segment granularity + * for this datasource has changed in the compaction config. + * + * @return true if the task was canceled, false otherwise. + */ + private boolean cancelTaskIfGranularityChanged( + ClientCompactionTaskQuery compactionTaskQuery, + DataSourceCompactionConfig dataSourceCompactionConfig + ) + { + if (dataSourceCompactionConfig == null + || dataSourceCompactionConfig.getGranularitySpec() == null + || compactionTaskQuery.getGranularitySpec() == null) { + return false; + } + + Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec() + .getSegmentGranularity(); + Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity(); + if (configuredSegmentGranularity.equals(taskSegmentGranularity)) { + return false; + } + + LOG.info( + "Cancelling task [%s] as task segmentGranularity is [%s] but compaction config segmentGranularity is [%s]", + compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity + ); + indexingServiceClient.cancelTask(compactionTaskQuery.getId()); + return true; } /** @@ -345,135 +322,184 @@ public class CompactSegments implements CoordinatorCustomDuty .collect(Collectors.toList()); } - private CoordinatorStats doRun( + private int getCompactionTaskCapacity(CoordinatorCompactionConfig dynamicConfig) + { + int totalWorkerCapacity; + try { + totalWorkerCapacity = dynamicConfig.isUseAutoScaleSlots() + ? indexingServiceClient.getTotalWorkerCapacityWithAutoScale() + : indexingServiceClient.getTotalWorkerCapacity(); + } + catch (Exception e) { + LOG.warn("Failed to get total worker capacity with auto scale slots. Falling back to current capacity count"); + totalWorkerCapacity = indexingServiceClient.getTotalWorkerCapacity(); + } + + return Math.min( + (int) (totalWorkerCapacity * dynamicConfig.getCompactionTaskSlotRatio()), + dynamicConfig.getMaxCompactionTaskSlots() + ); + } + + private int getAvailableCompactionTaskSlots(int compactionTaskCapacity, int busyCompactionTaskSlots) + { + final int availableCompactionTaskSlots; + if (busyCompactionTaskSlots > 0) { + availableCompactionTaskSlots = Math.max(0, compactionTaskCapacity - busyCompactionTaskSlots); + } else { + // compactionTaskCapacity might be 0 if totalWorkerCapacity is low. + // This guarantees that at least one slot is available if + // compaction is enabled and estimatedIncompleteCompactionTasks is 0. + availableCompactionTaskSlots = Math.max(1, compactionTaskCapacity); + } + LOG.info( + "Found [%d] available task slots for compaction out of max compaction task capacity [%d]", + availableCompactionTaskSlots, compactionTaskCapacity + ); + + return availableCompactionTaskSlots; + } + + /** + * Submits compaction tasks to the Overlord. Returns total number of tasks submitted. + */ + private int submitCompactionTasks( Map compactionConfigs, Map currentRunAutoCompactionSnapshotBuilders, int numAvailableCompactionTaskSlots, CompactionSegmentIterator iterator ) { + if (numAvailableCompactionTaskSlots <= 0) { + return 0; + } + int numSubmittedTasks = 0; int numCompactionTasksAndSubtasks = 0; while (iterator.hasNext() && numCompactionTasksAndSubtasks < numAvailableCompactionTaskSlots) { final List segmentsToCompact = iterator.next(); - - if (!segmentsToCompact.isEmpty()) { - final String dataSourceName = segmentsToCompact.get(0).getDataSource(); - // As these segments will be compacted, we will aggregates the statistic to the Compacted statistics - - AutoCompactionSnapshot.Builder snapshotBuilder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent( - dataSourceName, - k -> new AutoCompactionSnapshot.Builder(k, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING) - ); - snapshotBuilder.incrementBytesCompacted(segmentsToCompact.stream().mapToLong(DataSegment::getSize).sum()); - snapshotBuilder.incrementIntervalCountCompacted(segmentsToCompact.stream().map(DataSegment::getInterval).distinct().count()); - snapshotBuilder.incrementSegmentCountCompacted(segmentsToCompact.size()); - - final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); - - // Create granularitySpec to send to compaction task - ClientCompactionTaskGranularitySpec granularitySpec; - Granularity segmentGranularityToUse = null; - if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { - // Determines segmentGranularity from the segmentsToCompact - // Each batch of segmentToCompact from CompactionSegmentIterator will contains the same interval as - // segmentGranularity is not set in the compaction config - Interval interval = segmentsToCompact.get(0).getInterval(); - if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) { - try { - segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); - } - catch (IllegalArgumentException iae) { - // This case can happen if the existing segment interval result in complicated periods. - // Fall back to setting segmentGranularity as null - LOG.warn("Cannot determine segmentGranularity from interval [%s]", interval); - } - } else { - LOG.warn("segmentsToCompact does not have the same interval. Fallback to not setting segmentGranularity for auto compaction task"); - } - } else { - segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity(); - } - granularitySpec = new ClientCompactionTaskGranularitySpec( - segmentGranularityToUse, - config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null, - config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null - - ); - - // Create dimensionsSpec to send to compaction task - ClientCompactionTaskDimensionsSpec dimensionsSpec; - if (config.getDimensionsSpec() != null) { - dimensionsSpec = new ClientCompactionTaskDimensionsSpec( - config.getDimensionsSpec().getDimensions() - ); - } else { - dimensionsSpec = null; - } - - // Create transformSpec to send to compaction task - ClientCompactionTaskTransformSpec transformSpec = null; - if (config.getTransformSpec() != null) { - transformSpec = new ClientCompactionTaskTransformSpec( - config.getTransformSpec().getFilter() - ); - } - - Boolean dropExisting = null; - if (config.getIoConfig() != null) { - dropExisting = config.getIoConfig().isDropExisting(); - } - - // If all the segments found to be compacted are tombstones then dropExisting - // needs to be forced to true. This forcing needs to happen in the case that - // the flag is null, or it is false. It is needed when it is null to avoid the - // possibility of the code deciding to default it to false later. - // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to - // cover the tombstones found to be compacted as well as to mark them - // as compacted (update their lastCompactionState). If we don't force the - // flag then every time this compact duty runs it will find the same tombstones - // in the interval since their lastCompactionState - // was not set repeating this over and over and the duty will not make progress; it - // will become stuck on this set of tombstones. - // This forcing code should be revised - // when/if the autocompaction code policy to decide which segments to compact changes - if (dropExisting == null || !dropExisting) { - if (segmentsToCompact.stream().allMatch(dataSegment -> dataSegment.isTombstone())) { - dropExisting = true; - LOG.info("Forcing dropExisting to %s since all segments to compact are tombstones", dropExisting); - } - } - - // make tuningConfig - final String taskId = indexingServiceClient.compactSegments( - "coordinator-issued", - segmentsToCompact, - config.getTaskPriority(), - ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), config.getMetricsSpec() != null), - granularitySpec, - dimensionsSpec, - config.getMetricsSpec(), - transformSpec, - dropExisting, - newAutoCompactionContext(config.getTaskContext()) - ); - - LOG.info( - "Submitted a compactionTask[%s] for %s segments", - taskId, - segmentsToCompact.size() - ); - LOG.infoSegments(segmentsToCompact, "Compacting segments"); - // Count the compaction task itself + its sub tasks - numSubmittedTasks++; - numCompactionTasksAndSubtasks += findMaxNumTaskSlotsUsedByOneCompactionTask(config.getTuningConfig()); - } else { + if (segmentsToCompact.isEmpty()) { throw new ISE("segmentsToCompact is empty?"); } + + final String dataSourceName = segmentsToCompact.get(0).getDataSource(); + + // As these segments will be compacted, we will aggregates the statistic to the Compacted statistics + AutoCompactionSnapshot.Builder snapshotBuilder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent( + dataSourceName, + k -> new AutoCompactionSnapshot.Builder(k, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING) + ); + snapshotBuilder.incrementBytesCompacted( + segmentsToCompact.stream().mapToLong(DataSegment::getSize).sum() + ); + snapshotBuilder.incrementIntervalCountCompacted( + segmentsToCompact.stream() + .map(DataSegment::getInterval) + .distinct().count() + ); + snapshotBuilder.incrementSegmentCountCompacted(segmentsToCompact.size()); + + final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); + + // Create granularitySpec to send to compaction task + ClientCompactionTaskGranularitySpec granularitySpec; + Granularity segmentGranularityToUse = null; + if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { + // Determines segmentGranularity from the segmentsToCompact + // Each batch of segmentToCompact from CompactionSegmentIterator will contains the same interval as + // segmentGranularity is not set in the compaction config + Interval interval = segmentsToCompact.get(0).getInterval(); + if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) { + try { + segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); + } + catch (IllegalArgumentException iae) { + // This case can happen if the existing segment interval result in complicated periods. + // Fall back to setting segmentGranularity as null + LOG.warn("Cannot determine segmentGranularity from interval [%s]", interval); + } + } else { + LOG.warn( + "segmentsToCompact does not have the same interval. Fallback to not setting segmentGranularity for auto compaction task"); + } + } else { + segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity(); + } + granularitySpec = new ClientCompactionTaskGranularitySpec( + segmentGranularityToUse, + config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null, + config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null + ); + + // Create dimensionsSpec to send to compaction task + ClientCompactionTaskDimensionsSpec dimensionsSpec; + if (config.getDimensionsSpec() != null) { + dimensionsSpec = new ClientCompactionTaskDimensionsSpec( + config.getDimensionsSpec().getDimensions() + ); + } else { + dimensionsSpec = null; + } + + // Create transformSpec to send to compaction task + ClientCompactionTaskTransformSpec transformSpec = null; + if (config.getTransformSpec() != null) { + transformSpec = new ClientCompactionTaskTransformSpec( + config.getTransformSpec().getFilter() + ); + } + + Boolean dropExisting = null; + if (config.getIoConfig() != null) { + dropExisting = config.getIoConfig().isDropExisting(); + } + + // If all the segments found to be compacted are tombstones then dropExisting + // needs to be forced to true. This forcing needs to happen in the case that + // the flag is null, or it is false. It is needed when it is null to avoid the + // possibility of the code deciding to default it to false later. + // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to + // cover the tombstones found to be compacted as well as to mark them + // as compacted (update their lastCompactionState). If we don't force the + // flag then every time this compact duty runs it will find the same tombstones + // in the interval since their lastCompactionState + // was not set repeating this over and over and the duty will not make progress; it + // will become stuck on this set of tombstones. + // This forcing code should be revised + // when/if the autocompaction code policy to decide which segments to compact changes + if (dropExisting == null || !dropExisting) { + if (segmentsToCompact.stream().allMatch(DataSegment::isTombstone)) { + dropExisting = true; + LOG.info("Forcing dropExisting to true since all segments to compact are tombstones."); + } + } + + final String taskId = indexingServiceClient.compactSegments( + "coordinator-issued", + segmentsToCompact, + config.getTaskPriority(), + ClientCompactionTaskQueryTuningConfig.from( + config.getTuningConfig(), + config.getMaxRowsPerSegment(), + config.getMetricsSpec() != null + ), + granularitySpec, + dimensionsSpec, + config.getMetricsSpec(), + transformSpec, + dropExisting, + newAutoCompactionContext(config.getTaskContext()) + ); + + LOG.info("Submitted a compactionTask[%s] for [%d] segments", taskId, segmentsToCompact.size()); + LOG.infoSegments(segmentsToCompact, "Compacting segments"); + // Count the compaction task itself + its sub tasks + numSubmittedTasks++; + numCompactionTasksAndSubtasks += findMaxNumTaskSlotsUsedByOneCompactionTask(config.getTuningConfig()); } - return makeStats(currentRunAutoCompactionSnapshotBuilders, numSubmittedTasks, iterator); + return numSubmittedTasks; } private Map newAutoCompactionContext(@Nullable Map configuredContext) @@ -485,19 +511,13 @@ public class CompactSegments implements CoordinatorCustomDuty return newContext; } - private CoordinatorStats makeStats( + private void addCompactionSnapshotStats( Map currentRunAutoCompactionSnapshotBuilders, - int numCompactionTasks, - CompactionSegmentIterator iterator + CompactionSegmentIterator iterator, + CoordinatorRunStats stats ) { - final Map currentAutoCompactionSnapshotPerDataSource = new HashMap<>(); - final CoordinatorStats stats = new CoordinatorStats(); - stats.addToGlobalStat(COMPACTION_TASK_COUNT, numCompactionTasks); - - // Iterate through all the remaining segments in the iterator. - // As these segments could be compacted but were not compacted due to lack of task slot, we will aggregates - // the statistic to the AwaitingCompaction statistics + // Mark all the segments remaining in the iterator as "awaiting compaction" while (iterator.hasNext()) { final List segmentsToCompact = iterator.next(); if (!segmentsToCompact.isEmpty()) { @@ -549,7 +569,9 @@ public class CompactSegments implements CoordinatorCustomDuty builder.incrementIntervalCountSkipped(dataSourceSkippedStatistics.getSegmentIntervalCountSum()); } - for (Map.Entry autoCompactionSnapshotBuilderEntry : currentRunAutoCompactionSnapshotBuilders.entrySet()) { + final Map currentAutoCompactionSnapshotPerDataSource = new HashMap<>(); + for (Map.Entry autoCompactionSnapshotBuilderEntry + : currentRunAutoCompactionSnapshotBuilders.entrySet()) { final String dataSource = autoCompactionSnapshotBuilderEntry.getKey(); final AutoCompactionSnapshot.Builder builder = autoCompactionSnapshotBuilderEntry.getValue(); @@ -557,58 +579,31 @@ public class CompactSegments implements CoordinatorCustomDuty AutoCompactionSnapshot autoCompactionSnapshot = builder.build(); currentAutoCompactionSnapshotPerDataSource.put(dataSource, autoCompactionSnapshot); - // Use the complete snapshot to emits metrics - stats.addToDataSourceStat( - TOTAL_SIZE_OF_SEGMENTS_AWAITING, - dataSource, - autoCompactionSnapshot.getBytesAwaitingCompaction() - ); - stats.addToDataSourceStat( - TOTAL_COUNT_OF_SEGMENTS_AWAITING, - dataSource, - autoCompactionSnapshot.getSegmentCountAwaitingCompaction() - ); - stats.addToDataSourceStat( - TOTAL_INTERVAL_OF_SEGMENTS_AWAITING, - dataSource, - autoCompactionSnapshot.getIntervalCountAwaitingCompaction() - ); - stats.addToDataSourceStat( - TOTAL_SIZE_OF_SEGMENTS_COMPACTED, - dataSource, - autoCompactionSnapshot.getBytesCompacted() - ); - stats.addToDataSourceStat( - TOTAL_COUNT_OF_SEGMENTS_COMPACTED, - dataSource, - autoCompactionSnapshot.getSegmentCountCompacted() - ); - stats.addToDataSourceStat( - TOTAL_INTERVAL_OF_SEGMENTS_COMPACTED, - dataSource, - autoCompactionSnapshot.getIntervalCountCompacted() - ); - stats.addToDataSourceStat( - TOTAL_SIZE_OF_SEGMENTS_SKIPPED, - dataSource, - autoCompactionSnapshot.getBytesSkipped() - ); - stats.addToDataSourceStat( - TOTAL_COUNT_OF_SEGMENTS_SKIPPED, - dataSource, - autoCompactionSnapshot.getSegmentCountSkipped() - ); - stats.addToDataSourceStat( - TOTAL_INTERVAL_OF_SEGMENTS_SKIPPED, - dataSource, - autoCompactionSnapshot.getIntervalCountSkipped() - ); + // Use the complete snapshot to emit metrics + addStatsForDatasource(dataSource, autoCompactionSnapshot, stats); } // Atomic update of autoCompactionSnapshotPerDataSource with the latest from this coordinator run autoCompactionSnapshotPerDataSource.set(currentAutoCompactionSnapshotPerDataSource); + } - return stats; + private void addStatsForDatasource( + String dataSource, + AutoCompactionSnapshot autoCompactionSnapshot, + CoordinatorRunStats stats + ) + { + final RowKey rowKey = RowKey.forDatasource(dataSource); + + stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction()); + stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction()); + stats.add(Stats.Compaction.PENDING_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountAwaitingCompaction()); + stats.add(Stats.Compaction.COMPACTED_BYTES, rowKey, autoCompactionSnapshot.getBytesCompacted()); + stats.add(Stats.Compaction.COMPACTED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountCompacted()); + stats.add(Stats.Compaction.COMPACTED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountCompacted()); + stats.add(Stats.Compaction.SKIPPED_BYTES, rowKey, autoCompactionSnapshot.getBytesSkipped()); + stats.add(Stats.Compaction.SKIPPED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountSkipped()); + stats.add(Stats.Compaction.SKIPPED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountSkipped()); } @Nullable diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java deleted file mode 100644 index a6efbcb9316..00000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java +++ /dev/null @@ -1,525 +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.druid.server.coordinator.duty; - -import it.unimi.dsi.fastutil.objects.Object2IntMap; -import it.unimi.dsi.fastutil.objects.Object2LongMap; -import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.query.DruidMetrics; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidCluster; -import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.LoadQueuePeon; -import org.apache.druid.server.coordinator.ServerHolder; -import org.apache.druid.server.coordinator.rules.LoadRule; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.VersionedIntervalTimeline; - -/** - * Emits stats of the cluster and metrics of the coordination (including segment balancing) process. - */ -public class EmitClusterStatsAndMetrics implements CoordinatorDuty -{ - private static final Logger log = new Logger(EmitClusterStatsAndMetrics.class); - - public static final String TOTAL_CAPACITY = "totalCapacity"; - public static final String TOTAL_HISTORICAL_COUNT = "totalHistoricalCount"; - public static final String MAX_REPLICATION_FACTOR = "maxReplicationFactor"; - - private final DruidCoordinator coordinator; - private final String groupName; - private final boolean isContainCompactSegmentDuty; - - public EmitClusterStatsAndMetrics(DruidCoordinator coordinator, String groupName, boolean isContainCompactSegmentDuty) - { - this.coordinator = coordinator; - this.groupName = groupName; - this.isContainCompactSegmentDuty = isContainCompactSegmentDuty; - } - - private void emitTieredStat( - final ServiceEmitter emitter, - final String metricName, - final String tier, - final double value - ) - { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.TIER, tier) - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .build(metricName, value) - ); - } - - private void emitTieredStat( - final ServiceEmitter emitter, - final String metricName, - final String tier, - final long value - ) - { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.TIER, tier) - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .build(metricName, value) - ); - } - - private void emitTieredStats( - final ServiceEmitter emitter, - final String metricName, - final CoordinatorStats stats, - final String statName - ) - { - stats.forEachTieredStat( - statName, - (final String tier, final long count) -> { - emitTieredStat(emitter, metricName, tier, count); - } - ); - } - - private void emitDutyStat( - final ServiceEmitter emitter, - final String metricName, - final String duty, - final long value - ) - { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY, duty) - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .build(metricName, value) - ); - } - - private void emitDutyStats( - final ServiceEmitter emitter, - final String metricName, - final CoordinatorStats stats, - final String statName - ) - { - stats.forEachDutyStat( - statName, - (final String duty, final long count) -> { - emitDutyStat(emitter, metricName, duty, count); - } - ); - } - - @Override - public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) - { - DruidCluster cluster = params.getDruidCluster(); - CoordinatorStats stats = params.getCoordinatorStats(); - ServiceEmitter emitter = params.getEmitter(); - - if (DruidCoordinator.HISTORICAL_MANAGEMENT_DUTIES_DUTY_GROUP.equals(groupName)) { - emitStatsForHistoricalManagementDuties(cluster, stats, emitter, params); - } - if (isContainCompactSegmentDuty) { - emitStatsForCompactSegments(cluster, stats, emitter); - } - - // Emit coordinator runtime stats - emitDutyStats(emitter, "coordinator/time", stats, "runtime"); - - return params; - } - - private void emitStatsForHistoricalManagementDuties(DruidCluster cluster, CoordinatorStats stats, ServiceEmitter emitter, DruidCoordinatorRuntimeParams params) - { - stats.forEachTieredStat( - "assignedCount", - (final String tier, final long count) -> { - log.info( - "[%s] : Assigned %s segments among %,d servers", - tier, - count, - cluster.getHistoricalsByTier(tier).size() - ); - - emitTieredStat(emitter, "segment/assigned/count", tier, count); - } - ); - - stats.forEachTieredStat( - "droppedCount", - (final String tier, final long count) -> { - log.info( - "[%s] : Dropped %s segments among %,d servers", - tier, - count, - cluster.getHistoricalsByTier(tier).size() - ); - - emitTieredStat(emitter, "segment/dropped/count", tier, count); - } - ); - - emitTieredStats(emitter, "segment/cost/raw", stats, "initialCost"); - - emitTieredStats(emitter, "segment/cost/normalization", stats, "normalization"); - - emitTieredStats(emitter, "segment/moved/count", stats, "movedCount"); - emitTieredStats(emitter, "segment/unmoved/count", stats, "unmovedCount"); - - emitter.emit( - new ServiceMetricEvent.Builder() - .build( - "segment/deleted/count", - stats.getGlobalStat("deletedCount") - ) - ); - - stats.forEachTieredStat( - "normalizedInitialCostTimesOneThousand", - (final String tier, final long count) -> { - emitTieredStat(emitter, "segment/cost/normalized", tier, count / 1000d); - } - ); - - stats.forEachTieredStat( - "unneededCount", - (final String tier, final long count) -> { - log.info( - "[%s] : Removed %s unneeded segments among %,d servers", - tier, - count, - cluster.getHistoricalsByTier(tier).size() - ); - emitTieredStat(emitter, "segment/unneeded/count", tier, count); - } - ); - - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .build( - "segment/overShadowed/count", - stats.getGlobalStat("overShadowedCount") - ) - ); - - stats.forEachTieredStat( - "movedCount", - (final String tier, final long count) -> { - log.info("[%s] : Moved %,d segment(s)", tier, count); - } - ); - - stats.forEachTieredStat( - "unmovedCount", - (final String tier, final long count) -> { - log.info("[%s] : Let alone %,d segment(s)", tier, count); - } - ); - - log.info("Load Queues:"); - for (Iterable serverHolders : cluster.getSortedHistoricalsByTier()) { - for (ServerHolder serverHolder : serverHolders) { - ImmutableDruidServer server = serverHolder.getServer(); - LoadQueuePeon queuePeon = serverHolder.getPeon(); - log.info( - "Server[%s, %s, %s] has %,d left to load, %,d left to drop, %,d served, %,d bytes queued, %,d bytes served.", - server.getName(), - server.getType().toString(), - server.getTier(), - queuePeon.getSegmentsToLoad().size(), - queuePeon.getSegmentsToDrop().size(), - server.getNumSegments(), - queuePeon.getLoadQueueSize(), - server.getCurrSize() - ); - if (log.isDebugEnabled()) { - for (DataSegment segment : queuePeon.getSegmentsToLoad()) { - log.debug("Segment to load[%s]", segment); - } - for (DataSegment segment : queuePeon.getSegmentsToDrop()) { - log.debug("Segment to drop[%s]", segment); - } - } - stats.addToTieredStat(TOTAL_CAPACITY, server.getTier(), server.getMaxSize()); - stats.addToTieredStat(TOTAL_HISTORICAL_COUNT, server.getTier(), 1); - } - } - - - params.getDatabaseRuleManager() - .getAllRules() - .values() - .forEach( - rules -> rules.forEach( - rule -> { - if (rule instanceof LoadRule) { - ((LoadRule) rule).getTieredReplicants() - .forEach( - (tier, replica) -> stats.accumulateMaxTieredStat( - MAX_REPLICATION_FACTOR, - tier, - replica - )); - } - } - )); - - emitTieredStats(emitter, "tier/required/capacity", stats, LoadRule.REQUIRED_CAPACITY); - emitTieredStats(emitter, "tier/total/capacity", stats, TOTAL_CAPACITY); - - emitTieredStats(emitter, "tier/replication/factor", stats, MAX_REPLICATION_FACTOR); - emitTieredStats(emitter, "tier/historical/count", stats, TOTAL_HISTORICAL_COUNT); - - // Emit coordinator metrics - params - .getLoadManagementPeons() - .forEach((final String serverName, final LoadQueuePeon queuePeon) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/loadQueue/size", queuePeon.getLoadQueueSize() - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/loadQueue/failed", queuePeon.getAndResetFailedAssignCount() - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/loadQueue/count", queuePeon.getSegmentsToLoad().size() - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/dropQueue/count", queuePeon.getSegmentsToDrop().size() - ) - ); - }); - - coordinator.computeNumsUnavailableUsedSegmentsPerDataSource().object2IntEntrySet().forEach( - (final Object2IntMap.Entry entry) -> { - final String dataSource = entry.getKey(); - final int numUnavailableUsedSegmentsInDataSource = entry.getIntValue(); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource).build( - "segment/unavailable/count", numUnavailableUsedSegmentsInDataSource - ) - ); - } - ); - - coordinator.computeUnderReplicationCountsPerDataSourcePerTier().forEach( - (final String tier, final Object2LongMap underReplicationCountsPerDataSource) -> { - for (final Object2LongMap.Entry entry : underReplicationCountsPerDataSource.object2LongEntrySet()) { - final String dataSource = entry.getKey(); - final long underReplicationCount = entry.getLongValue(); - - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.TIER, tier) - .setDimension(DruidMetrics.DATASOURCE, dataSource).build( - "segment/underReplicated/count", underReplicationCount - ) - ); - } - } - ); - - // Emit segment metrics - params.getUsedSegmentsTimelinesPerDataSource().forEach( - (String dataSource, VersionedIntervalTimeline dataSourceWithUsedSegments) -> { - long totalSizeOfUsedSegments = dataSourceWithUsedSegments - .iterateAllObjects() - .stream() - .mapToLong(DataSegment::getSize) - .sum(); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("segment/size", totalSizeOfUsedSegments) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("segment/count", dataSourceWithUsedSegments.getNumObjects()) - ); - } - ); - } - - private void emitStatsForCompactSegments(DruidCluster cluster, CoordinatorStats stats, ServiceEmitter emitter) - { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .build( - "compact/task/count", - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) - ) - ); - - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .build( - "compactTask/maxSlot/count", - stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT) - ) - ); - - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .build( - "compactTask/availableSlot/count", - stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT) - ) - ); - - stats.forEachDataSourceStat( - CompactSegments.TOTAL_SIZE_OF_SEGMENTS_AWAITING, - (final String dataSource, final long count) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("segment/waitCompact/bytes", count) - ); - } - ); - - stats.forEachDataSourceStat( - CompactSegments.TOTAL_COUNT_OF_SEGMENTS_AWAITING, - (final String dataSource, final long count) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("segment/waitCompact/count", count) - ); - } - ); - - stats.forEachDataSourceStat( - CompactSegments.TOTAL_INTERVAL_OF_SEGMENTS_AWAITING, - (final String dataSource, final long count) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("interval/waitCompact/count", count) - ); - } - ); - - stats.forEachDataSourceStat( - CompactSegments.TOTAL_SIZE_OF_SEGMENTS_SKIPPED, - (final String dataSource, final long count) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("segment/skipCompact/bytes", count) - ); - } - ); - - stats.forEachDataSourceStat( - CompactSegments.TOTAL_COUNT_OF_SEGMENTS_SKIPPED, - (final String dataSource, final long count) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("segment/skipCompact/count", count) - ); - } - ); - - stats.forEachDataSourceStat( - CompactSegments.TOTAL_INTERVAL_OF_SEGMENTS_SKIPPED, - (final String dataSource, final long count) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("interval/skipCompact/count", count) - ); - } - ); - - stats.forEachDataSourceStat( - CompactSegments.TOTAL_SIZE_OF_SEGMENTS_COMPACTED, - (final String dataSource, final long count) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("segment/compacted/bytes", count) - ); - } - ); - - stats.forEachDataSourceStat( - CompactSegments.TOTAL_COUNT_OF_SEGMENTS_COMPACTED, - (final String dataSource, final long count) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("segment/compacted/count", count) - ); - } - ); - - stats.forEachDataSourceStat( - CompactSegments.TOTAL_INTERVAL_OF_SEGMENTS_COMPACTED, - (final String dataSource, final long count) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DUTY_GROUP, groupName) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .build("interval/compacted/count", count) - ); - } - ); - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/LogUsedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/LogUsedSegments.java deleted file mode 100644 index 3904db9ecd5..00000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/LogUsedSegments.java +++ /dev/null @@ -1,53 +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.druid.server.coordinator.duty; - -import org.apache.druid.client.DataSourcesSnapshot; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.timeline.DataSegment; - -public class LogUsedSegments implements CoordinatorDuty -{ - private static final EmittingLogger log = new EmittingLogger(LogUsedSegments.class); - - public LogUsedSegments() - { - } - - @Override - public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) - { - log.debug("Starting coordination. Getting used segments."); - - // Log info about all used segments only if debug logging is enabled - if (log.isDebugEnabled()) { - DataSourcesSnapshot dataSourcesSnapshot = params.getDataSourcesSnapshot(); - log.debug("Used Segments"); - for (DataSegment dataSegment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) { - log.debug(" %s", dataSegment); - } - } - - log.info("Found [%,d] used segments.", params.getUsedSegments().size()); - - return params; - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnused.java similarity index 63% rename from server/src/main/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegments.java rename to server/src/main/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnused.java index 1e700248a29..1a63083bf77 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnused.java @@ -21,29 +21,32 @@ package org.apache.druid.server.coordinator.duty; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentTimeline; +import org.joda.time.DateTime; +import java.time.Duration; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.SortedSet; -public class MarkAsUnusedOvershadowedSegments implements CoordinatorDuty +public class MarkOvershadowedSegmentsAsUnused implements CoordinatorDuty { - private static final Logger log = new Logger(MarkAsUnusedOvershadowedSegments.class); + private static final Logger log = new Logger(MarkOvershadowedSegmentsAsUnused.class); private final DruidCoordinator coordinator; - public MarkAsUnusedOvershadowedSegments(DruidCoordinator coordinator) + public MarkOvershadowedSegmentsAsUnused(DruidCoordinator coordinator) { this.coordinator = coordinator; } @@ -51,9 +54,15 @@ public class MarkAsUnusedOvershadowedSegments implements CoordinatorDuty @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - // Mark as unused overshadowed segments only if we've had enough time to make sure we aren't flapping with old data. - if (!params.coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements()) { - log.info("Skipping MarkAsUnused as coordinator is not leading enough time."); + // Mark overshadowed segments as unused only if the coordinator has been running + // long enough to have refreshed its metadata view + final DateTime coordinatorStartTime = params.getCoordinatorStartTime(); + final long delayMillis = params.getCoordinatorDynamicConfig().getMarkSegmentAsUnusedDelayMillis(); + if (DateTimes.nowUtc().isBefore(coordinatorStartTime.plus(delayMillis))) { + log.info( + "Skipping MarkAsUnused until [%s] have elapsed after coordinator start [%s].", + Duration.ofMillis(delayMillis), coordinatorStartTime + ); return params; } @@ -63,37 +72,40 @@ public class MarkAsUnusedOvershadowedSegments implements CoordinatorDuty return params; } - CoordinatorStats stats = new CoordinatorStats(); - DruidCluster cluster = params.getDruidCluster(); final Map timelines = new HashMap<>(); - for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { - for (ServerHolder serverHolder : serverHolders) { - addSegmentsFromServer(serverHolder, timelines); - } - } - - for (ServerHolder serverHolder : cluster.getBrokers()) { - addSegmentsFromServer(serverHolder, timelines); - } + cluster.getHistoricals().values().forEach( + historicals -> historicals.forEach( + historical -> addSegmentsFromServer(historical, timelines) + ) + ); + cluster.getBrokers().forEach( + broker -> addSegmentsFromServer(broker, timelines) + ); // Note that we do not include segments from ingestion services such as tasks or indexers, // to prevent unpublished segments from prematurely overshadowing segments. - // Mark all segments as unused in db that are overshadowed by served segments + // Mark all segments overshadowed by served segments as unused final Map> datasourceToUnusedSegments = new HashMap<>(); for (DataSegment dataSegment : allOvershadowedSegments) { SegmentTimeline timeline = timelines.get(dataSegment.getDataSource()); if (timeline != null && timeline.isOvershadowed(dataSegment)) { datasourceToUnusedSegments.computeIfAbsent(dataSegment.getDataSource(), ds -> new HashSet<>()) .add(dataSegment.getId()); - stats.addToGlobalStat("overShadowedCount", 1); } } - datasourceToUnusedSegments.forEach(coordinator::markSegmentsAsUnused); - return params.buildFromExisting().withCoordinatorStats(stats).build(); + final CoordinatorRunStats stats = params.getCoordinatorStats(); + datasourceToUnusedSegments.forEach( + (datasource, unusedSegments) -> { + stats.addToDatasourceStat(Stats.Segments.OVERSHADOWED, datasource, unusedSegments.size()); + coordinator.markSegmentsAsUnused(datasource, unusedSegments); + } + ); + + return params; } private void addSegmentsFromServer( @@ -104,9 +116,9 @@ public class MarkAsUnusedOvershadowedSegments implements CoordinatorDuty ImmutableDruidServer server = serverHolder.getServer(); for (ImmutableDruidDataSource dataSource : server.getDataSources()) { - SegmentTimeline timeline = timelines - .computeIfAbsent(dataSource.getName(), dsName -> new SegmentTimeline()); - timeline.addSegments(dataSource.getSegments().iterator()); + timelines + .computeIfAbsent(dataSource.getName(), dsName -> new SegmentTimeline()) + .addSegments(dataSource.getSegments().iterator()); } } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java index 6a79fd9dcaf..eb037d1212f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java @@ -24,64 +24,39 @@ import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCluster; -import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.ReplicationThrottler; +import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; import org.apache.druid.server.coordinator.rules.BroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; -import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.stream.Collectors; /** + * Duty to run retention rules. + *

+ * The params returned from {@code run()} must have these fields initialized: + *

    + *
  • {@link DruidCoordinatorRuntimeParams#getBroadcastDatasources()}
  • + *
+ * These fields are used by the downstream coordinator duty, {@link BalanceSegments}. */ public class RunRules implements CoordinatorDuty { private static final EmittingLogger log = new EmittingLogger(RunRules.class); private static final int MAX_MISSING_RULES = 10; - private final ReplicationThrottler replicatorThrottler; - - private final DruidCoordinator coordinator; - - public RunRules(DruidCoordinator coordinator) - { - this( - new ReplicationThrottler( - coordinator.getDynamicConfigs().getReplicationThrottleLimit(), - coordinator.getDynamicConfigs().getReplicantLifetime(), - false - ), - coordinator - ); - } - - public RunRules(ReplicationThrottler replicatorThrottler, DruidCoordinator coordinator) - { - this.replicatorThrottler = replicatorThrottler; - this.coordinator = coordinator; - } - @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - replicatorThrottler.updateParams( - coordinator.getDynamicConfigs().getReplicationThrottleLimit(), - coordinator.getDynamicConfigs().getReplicantLifetime(), - false - ); - - CoordinatorStats stats = new CoordinatorStats(); - DruidCluster cluster = params.getDruidCluster(); - + final DruidCluster cluster = params.getDruidCluster(); if (cluster.isEmpty()) { - log.warn("Uh... I have no servers. Not assigning anything..."); + log.warn("Cluster has no servers. Not running any rules."); return params; } @@ -89,60 +64,31 @@ public class RunRules implements CoordinatorDuty // eventually will be unloaded from Historical servers. Segments overshadowed by *served* used segments are marked // as unused in MarkAsUnusedOvershadowedSegments, and then eventually Coordinator sends commands to Historical nodes // to unload such segments in UnloadUnusedSegments. - Set overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments(); + final Set overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments(); + final Set usedSegments = params.getUsedSegments(); + log.info( + "Applying retention rules on [%d] used segments, skipping [%d] overshadowed segments.", + usedSegments.size(), overshadowed.size() + ); - for (String tier : cluster.getTierNames()) { - replicatorThrottler.updateReplicationState(tier); - } + final StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); + final MetadataRuleManager databaseRuleManager = params.getDatabaseRuleManager(); - DruidCoordinatorRuntimeParams paramsWithReplicationManager = params - .buildFromExistingWithoutSegmentsMetadata() - .withReplicationManager(replicatorThrottler) - .build(); + int missingRules = 0; + final DateTime now = DateTimes.nowUtc(); + final List segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES); // Run through all matched rules for used segments - DateTime now = DateTimes.nowUtc(); - MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager(); - - final List segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES); - int missingRules = 0; - - final Set broadcastDatasources = new HashSet<>(); - for (ImmutableDruidDataSource dataSource : params.getDataSourcesSnapshot().getDataSourcesMap().values()) { - List rules = databaseRuleManager.getRulesWithDefault(dataSource.getName()); - for (Rule rule : rules) { - // A datasource is considered a broadcast datasource if it has any broadcast rules. - // The set of broadcast datasources is used by BalanceSegments, so it's important that RunRules - // executes before BalanceSegments. - if (rule instanceof BroadcastDistributionRule) { - broadcastDatasources.add(dataSource.getName()); - break; - } - } - } - - for (DataSegment segment : params.getUsedSegments()) { + for (DataSegment segment : usedSegments) { if (overshadowed.contains(segment)) { - // Skipping overshadowed segments + // Skip overshadowed segments continue; } List rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource()); boolean foundMatchingRule = false; for (Rule rule : rules) { if (rule.appliesTo(segment, now)) { - if ( - stats.getGlobalStat( - "totalNonPrimaryReplicantsLoaded") >= paramsWithReplicationManager.getCoordinatorDynamicConfig() - .getMaxNonPrimaryReplicantsToLoad() - && !paramsWithReplicationManager.getReplicationManager().isLoadPrimaryReplicantsOnly() - ) { - log.info( - "Maximum number of non-primary replicants [%d] have been loaded for the current RunRules execution. Only loading primary replicants from here on for this coordinator run cycle.", - paramsWithReplicationManager.getCoordinatorDynamicConfig().getMaxNonPrimaryReplicantsToLoad() - ); - paramsWithReplicationManager.getReplicationManager().setLoadPrimaryReplicantsOnly(true); - } - stats.accumulate(rule.run(coordinator, paramsWithReplicationManager, segment)); + rule.run(segment, segmentAssigner); foundMatchingRule = true; break; } @@ -164,8 +110,36 @@ public class RunRules implements CoordinatorDuty } return params.buildFromExisting() - .withCoordinatorStats(stats) - .withBroadcastDatasources(broadcastDatasources) + .withBroadcastDatasources(getBroadcastDatasources(params)) .build(); } + + private Set getBroadcastDatasources(DruidCoordinatorRuntimeParams params) + { + final Set broadcastDatasources = + params.getDataSourcesSnapshot().getDataSourcesMap().values().stream() + .map(ImmutableDruidDataSource::getName) + .filter(datasource -> isBroadcastDatasource(datasource, params)) + .collect(Collectors.toSet()); + + if (!broadcastDatasources.isEmpty()) { + log.info("Found broadcast datasources [%s] which will not participate in balancing.", broadcastDatasources); + } + + return broadcastDatasources; + } + + /** + * A datasource is considered a broadcast datasource if it has even one + * Broadcast Rule. Segments of broadcast datasources: + *
    + *
  • Do not participate in balancing
  • + *
  • Are unloaded if unused, even from realtime servers
  • + *
+ */ + private boolean isBroadcastDatasource(String datasource, DruidCoordinatorRuntimeParams params) + { + return params.getDatabaseRuleManager().getRulesWithDefault(datasource).stream() + .anyMatch(rule -> rule instanceof BroadcastDistributionRule); + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/SegmentCompactionUtil.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/SegmentCompactionUtil.java deleted file mode 100644 index e9e57320622..00000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/SegmentCompactionUtil.java +++ /dev/null @@ -1,49 +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.druid.server.coordinator.duty; - -import com.google.common.base.Preconditions; -import org.joda.time.Interval; - -/** - * Util class used by {@link CompactSegments} and {@link CompactionSegmentSearchPolicy}. - */ -class SegmentCompactionUtil -{ - /** - * Removes {@code smallInterval} from {@code largeInterval}. The end of both intervals should be same. - * - * @return an interval of {@code largeInterval} - {@code smallInterval}. - */ - static Interval removeIntervalFromEnd(Interval largeInterval, Interval smallInterval) - { - Preconditions.checkArgument( - largeInterval.getEnd().equals(smallInterval.getEnd()), - "end should be same. largeInterval[%s] smallInterval[%s]", - largeInterval, - smallInterval - ); - return new Interval(largeInterval.getStart(), smallInterval.getStart()); - } - - private SegmentCompactionUtil() - { - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegments.java index 75bcb8fe175..7e7301cd170 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegments.java @@ -22,20 +22,17 @@ package org.apache.druid.server.coordinator.duty; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.rules.BroadcastDistributionRule; -import org.apache.druid.server.coordinator.rules.Rule; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Set; -import java.util.SortedSet; /** * Unloads segments that are no longer marked as used from servers. @@ -44,62 +41,38 @@ public class UnloadUnusedSegments implements CoordinatorDuty { private static final Logger log = new Logger(UnloadUnusedSegments.class); + private final SegmentLoadQueueManager loadQueueManager; + + public UnloadUnusedSegments(SegmentLoadQueueManager loadQueueManager) + { + this.loadQueueManager = loadQueueManager; + } + @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - CoordinatorStats stats = new CoordinatorStats(); - Set usedSegments = params.getUsedSegments(); - DruidCluster cluster = params.getDruidCluster(); - - Map broadcastStatusByDatasource = new HashMap<>(); + final Map broadcastStatusByDatasource = new HashMap<>(); for (String broadcastDatasource : params.getBroadcastDatasources()) { broadcastStatusByDatasource.put(broadcastDatasource, true); } - for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { - for (ServerHolder serverHolder : serverHolders) { - handleUnusedSegmentsForServer( - serverHolder, - usedSegments, + final CoordinatorRunStats stats = params.getCoordinatorStats(); + params.getDruidCluster().getAllServers().forEach( + server -> handleUnusedSegmentsForServer( + server, params, stats, - false, broadcastStatusByDatasource - ); - } - } + ) + ); - for (ServerHolder serverHolder : cluster.getBrokers()) { - handleUnusedSegmentsForServer( - serverHolder, - usedSegments, - params, - stats, - false, - broadcastStatusByDatasource - ); - } - - for (ServerHolder serverHolder : cluster.getRealtimes()) { - handleUnusedSegmentsForServer( - serverHolder, - usedSegments, - params, - stats, - true, - broadcastStatusByDatasource - ); - } - - return params.buildFromExisting().withCoordinatorStats(stats).build(); + return params; } private void handleUnusedSegmentsForServer( ServerHolder serverHolder, - Set usedSegments, DruidCoordinatorRuntimeParams params, - CoordinatorStats stats, - boolean dropBroadcastOnly, + CoordinatorRunStats stats, Map broadcastStatusByDatasource ) { @@ -107,16 +80,7 @@ public class UnloadUnusedSegments implements CoordinatorDuty for (ImmutableDruidDataSource dataSource : server.getDataSources()) { boolean isBroadcastDatasource = broadcastStatusByDatasource.computeIfAbsent( dataSource.getName(), - (dataSourceName) -> { - List rules = params.getDatabaseRuleManager().getRulesWithDefault(dataSource.getName()); - for (Rule rule : rules) { - // A datasource is considered a broadcast datasource if it has any broadcast rules. - if (rule instanceof BroadcastDistributionRule) { - return true; - } - } - return false; - } + dataSourceName -> isBroadcastDatasource(dataSourceName, params) ); // The coordinator tracks used segments by examining the metadata store. @@ -127,26 +91,35 @@ public class UnloadUnusedSegments implements CoordinatorDuty // datasource, this will result in the those segments not being dropped from tasks. // A more robust solution which requires a larger rework could be to expose // the set of segments that were created by a task/indexer here, and exclude them. - if (dropBroadcastOnly && !isBroadcastDatasource) { + if (serverHolder.isRealtimeServer() && !isBroadcastDatasource) { continue; } + int totalUnneededCount = 0; + final Set usedSegments = params.getUsedSegments(); for (DataSegment segment : dataSource.getSegments()) { - if (!usedSegments.contains(segment)) { - LoadQueuePeon queuePeon = params.getLoadManagementPeons().get(server.getName()); - - if (!queuePeon.getSegmentsToDrop().contains(segment)) { - queuePeon.dropSegment(segment, success -> {}); - stats.addToTieredStat("unneededCount", server.getTier(), 1); - log.info( - "Dropping uneeded segment [%s] from server [%s] in tier [%s]", - segment.getId(), - server.getName(), - server.getTier() - ); - } + if (!usedSegments.contains(segment) + && loadQueueManager.dropSegment(segment, serverHolder)) { + totalUnneededCount++; + log.info( + "Dropping uneeded segment [%s] from server [%s] in tier [%s]", + segment.getId(), server.getName(), server.getTier() + ); } } + + if (totalUnneededCount > 0) { + stats.addToSegmentStat(Stats.Segments.UNNEEDED, server.getTier(), dataSource.getName(), totalUnneededCount); + } } } + + /** + * A datasource is considered a broadcast datasource if it has even one broadcast rule. + */ + private boolean isBroadcastDatasource(String datasource, DruidCoordinatorRuntimeParams params) + { + return params.getDatabaseRuleManager().getRulesWithDefault(datasource).stream() + .anyMatch(rule -> rule instanceof BroadcastDistributionRule); + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java similarity index 77% rename from server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java rename to server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java index 24c96e19d23..2303be26d8b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/CuratorLoadQueuePeon.java @@ -17,20 +17,20 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.loading; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.api.CuratorWatcher; import org.apache.curator.utils.ZKPaths; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.coordination.DataSegmentChangeRequest; -import org.apache.druid.server.coordination.SegmentChangeRequestDrop; -import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.SegmentChangeRequestNoop; +import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.DruidCoordinatorConfig; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -38,11 +38,7 @@ import org.apache.zookeeper.data.Stat; import javax.annotation.Nonnull; import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; +import java.util.HashSet; import java.util.Set; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListSet; @@ -50,7 +46,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; /** @@ -63,7 +58,7 @@ import java.util.concurrent.atomic.AtomicLong; * of the same or different methods. */ @Deprecated -public class CuratorLoadQueuePeon extends LoadQueuePeon +public class CuratorLoadQueuePeon implements LoadQueuePeon { private static final EmittingLogger log = new EmittingLogger(CuratorLoadQueuePeon.class); @@ -80,45 +75,42 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon private final DruidCoordinatorConfig config; private final AtomicLong queuedSize = new AtomicLong(0); - private final AtomicInteger failedAssignCount = new AtomicInteger(0); + private final CoordinatorRunStats stats = new CoordinatorRunStats(); /** * Needs to be thread safe since it can be concurrently accessed via - * {@link #loadSegment(DataSegment, LoadPeonCallback)}, {@link #actionCompleted(SegmentHolder)}, - * {@link #getSegmentsToLoad()} and {@link #stop()} + * {@link #loadSegment(DataSegment, SegmentAction, LoadPeonCallback)}, + * {@link #actionCompleted(SegmentHolder)}, {@link #getSegmentsToLoad()} and + * {@link #stop()}. */ - private final ConcurrentSkipListMap segmentsToLoad = new ConcurrentSkipListMap<>( - DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST - ); + private final ConcurrentSkipListMap segmentsToLoad + = new ConcurrentSkipListMap<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST); /** * Needs to be thread safe since it can be concurrently accessed via * {@link #dropSegment(DataSegment, LoadPeonCallback)}, {@link #actionCompleted(SegmentHolder)}, * {@link #getSegmentsToDrop()} and {@link #stop()} */ - private final ConcurrentSkipListMap segmentsToDrop = new ConcurrentSkipListMap<>( - DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST - ); + private final ConcurrentSkipListMap segmentsToDrop + = new ConcurrentSkipListMap<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST); /** * Needs to be thread safe since it can be concurrently accessed via * {@link #markSegmentToDrop(DataSegment)}}, {@link #unmarkSegmentToDrop(DataSegment)}} * and {@link #getSegmentsToDrop()} */ - private final ConcurrentSkipListSet segmentsMarkedToDrop = new ConcurrentSkipListSet<>( - DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST - ); + private final ConcurrentSkipListSet segmentsMarkedToDrop + = new ConcurrentSkipListSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST); /** * Needs to be thread safe since it can be concurrently accessed via * {@link #failAssign(SegmentHolder, boolean, Exception)}, {@link #actionCompleted(SegmentHolder)}, * {@link #getTimedOutSegments()} and {@link #stop()} */ - private final ConcurrentSkipListSet timedOutSegments = new ConcurrentSkipListSet<>( - DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST - ); + private final ConcurrentSkipListSet timedOutSegments = + new ConcurrentSkipListSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST); - CuratorLoadQueuePeon( + public CuratorLoadQueuePeon( CuratorFramework curator, String basePath, ObjectMapper jsonMapper, @@ -142,6 +134,15 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon return segmentsToLoad.keySet(); } + @Override + public Set getSegmentsInQueue() + { + final Set segmentsInQueue = new HashSet<>(); + segmentsInQueue.addAll(segmentsToLoad.values()); + segmentsInQueue.addAll(segmentsToDrop.values()); + return segmentsInQueue; + } + @JsonProperty @Override public Set getSegmentsToDrop() @@ -163,27 +164,21 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon } @Override - public long getLoadQueueSize() + public long getSizeOfSegmentsToLoad() { return queuedSize.get(); } @Override - public int getAndResetFailedAssignCount() + public CoordinatorRunStats getAndResetStats() { - return failedAssignCount.getAndSet(0); + return stats.getSnapshotAndReset(); } @Override - public int getNumberOfSegmentsInQueue() + public void loadSegment(final DataSegment segment, SegmentAction action, @Nullable final LoadPeonCallback callback) { - return segmentsToLoad.size(); - } - - @Override - public void loadSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback) - { - SegmentHolder segmentHolder = new SegmentHolder(segment, Action.LOAD, Collections.singletonList(callback)); + SegmentHolder segmentHolder = new SegmentHolder(segment, action, callback); final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { existingHolder.addCallback(callback); @@ -197,7 +192,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon @Override public void dropSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback) { - SegmentHolder segmentHolder = new SegmentHolder(segment, Action.DROP, Collections.singletonList(callback)); + SegmentHolder segmentHolder = new SegmentHolder(segment, SegmentAction.DROP, callback); final SegmentHolder existingHolder = segmentsToDrop.putIfAbsent(segment, segmentHolder); if (existingHolder != null) { existingHolder.addCallback(callback); @@ -324,11 +319,13 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon { switch (segmentHolder.getAction()) { case LOAD: + case REPLICATE: + case MOVE_TO: // When load failed a segment will be removed from the segmentsToLoad twice and // null value will be returned at the second time in which case queueSize may be negative. // See https://github.com/apache/druid/pull/10362 for more details. if (null != segmentsToLoad.remove(segmentHolder.getSegment())) { - queuedSize.addAndGet(-segmentHolder.getSegmentSize()); + queuedSize.addAndGet(-segmentHolder.getSegment().getSize()); timedOutSegments.remove(segmentHolder.getSegment()); } break; @@ -363,7 +360,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon timedOutSegments.clear(); queuedSize.set(0L); - failedAssignCount.set(0); + stats.clear(); } private void onZkNodeDeleted(SegmentHolder segmentHolder, String path) @@ -391,7 +388,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon if (e != null) { log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, segmentHolder); } - failedAssignCount.getAndIncrement(); + stats.add(Stats.SegmentQueue.FAILED_ACTIONS, 1); if (handleTimeout) { // Avoid removing the segment entry from the load/drop list in case config.getLoadTimeoutDelay() expires. @@ -409,91 +406,15 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon } } - private enum Action + @Override + public boolean cancelOperation(DataSegment segment) { - LOAD, DROP - } - - private static class SegmentHolder - { - private final DataSegment segment; - private final DataSegmentChangeRequest changeRequest; - private final Action type; - // Guaranteed to store only non-null elements - private final List callbacks = new ArrayList<>(); - - private SegmentHolder( - DataSegment segment, - Action type, - Collection callbacksParam - ) - { - this.segment = segment; - this.type = type; - this.changeRequest = (type == Action.LOAD) - ? new SegmentChangeRequestLoad(segment) - : new SegmentChangeRequestDrop(segment); - Iterator itr = callbacksParam.iterator(); - while (itr.hasNext()) { - LoadPeonCallback c = itr.next(); - if (c != null) { - callbacks.add(c); - } - } - } - - public DataSegment getSegment() - { - return segment; - } - - public Action getAction() - { - return type; - } - - public String getSegmentIdentifier() - { - return segment.getId().toString(); - } - - public long getSegmentSize() - { - return segment.getSize(); - } - - public void addCallback(@Nullable LoadPeonCallback newCallback) - { - if (newCallback != null) { - synchronized (callbacks) { - callbacks.add(newCallback); - } - } - } - - List snapshotCallbacks() - { - synchronized (callbacks) { - // Return an immutable copy so that callers don't have to worry about concurrent modification - return ImmutableList.copyOf(callbacks); - } - } - - public DataSegmentChangeRequest getChangeRequest() - { - return changeRequest; - } - - @Override - public String toString() - { - return changeRequest.toString(); - } + return false; } private void executeCallbacks(SegmentHolder holder, boolean success) { - for (LoadPeonCallback callback : holder.snapshotCallbacks()) { + for (LoadPeonCallback callback : holder.getCallbacks()) { callBackExecutor.submit(() -> callback.execute(success)); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java similarity index 65% rename from server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java rename to server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index e970c56c4ea..9c133fce8c6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -17,12 +17,11 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.loading; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; -import com.google.common.collect.Iterators; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -36,9 +35,12 @@ import org.apache.druid.java.util.http.client.Request; import org.apache.druid.server.coordination.DataSegmentChangeCallback; import org.apache.druid.server.coordination.DataSegmentChangeHandler; import org.apache.druid.server.coordination.DataSegmentChangeRequest; -import org.apache.druid.server.coordination.SegmentChangeRequestDrop; -import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.SegmentLoadDropHandler; +import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; +import org.apache.druid.server.coordinator.DruidCoordinatorConfig; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.CoordinatorStat; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -51,21 +53,22 @@ import java.net.MalformedURLException; import java.net.URL; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.ConcurrentSkipListSet; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; /** + * */ -public class HttpLoadQueuePeon extends LoadQueuePeon +public class HttpLoadQueuePeon implements LoadQueuePeon { public static final TypeReference> REQUEST_ENTITY_TYPE_REF = new TypeReference>() @@ -80,17 +83,24 @@ public class HttpLoadQueuePeon extends LoadQueuePeon private static final EmittingLogger log = new EmittingLogger(HttpLoadQueuePeon.class); private final AtomicLong queuedSize = new AtomicLong(0); - private final AtomicInteger failedAssignCount = new AtomicInteger(0); + private final CoordinatorRunStats stats = new CoordinatorRunStats(); - private final ConcurrentSkipListMap segmentsToLoad = new ConcurrentSkipListMap<>( - DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST - ); - private final ConcurrentSkipListMap segmentsToDrop = new ConcurrentSkipListMap<>( - DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST - ); - private final ConcurrentSkipListSet segmentsMarkedToDrop = new ConcurrentSkipListSet<>( - DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST - ); + private final ConcurrentMap segmentsToLoad = new ConcurrentHashMap<>(); + private final ConcurrentMap segmentsToDrop = new ConcurrentHashMap<>(); + private final Set segmentsMarkedToDrop = ConcurrentHashMap.newKeySet(); + + /** + * Segments currently in queue ordered by priority and interval. This includes + * drop requests as well. This need not be thread-safe as all operations on it + * are synchronized with the {@link #lock}. + */ + private final Set queuedSegments = new TreeSet<>(); + + /** + * Set of segments for which requests have been sent to the server and can + * not be cancelled anymore. This need not be thread-safe. + */ + private final Set activeRequestSegments = new HashSet<>(); private final ScheduledExecutorService processingExecutor; @@ -153,18 +163,25 @@ public class HttpLoadQueuePeon extends LoadQueuePeon final List newRequests = new ArrayList<>(batchSize); synchronized (lock) { - Iterator> iter = Iterators.concat( - segmentsToDrop.entrySet().iterator(), - segmentsToLoad.entrySet().iterator() - ); + final Iterator queuedSegmentIterator = queuedSegments.iterator(); - while (newRequests.size() < batchSize && iter.hasNext()) { - Map.Entry entry = iter.next(); - if (entry.getValue().hasTimedOut()) { - entry.getValue().requestFailed("timed out"); - iter.remove(); + final long currentTimeMillis = System.currentTimeMillis(); + while (newRequests.size() < batchSize && queuedSegmentIterator.hasNext()) { + final SegmentHolder holder = queuedSegmentIterator.next(); + final DataSegment segment = holder.getSegment(); + if (hasRequestTimedOut(holder, currentTimeMillis)) { + onRequestFailed(holder, "timed out"); + queuedSegmentIterator.remove(); + if (holder.isLoad()) { + segmentsToLoad.remove(segment); + } else { + segmentsToDrop.remove(segment); + } + activeRequestSegments.remove(segment); } else { - newRequests.add(entry.getValue().getChangeRequest()); + newRequests.add(holder.getChangeRequest()); + holder.markRequestSentToServer(); + activeRequestSegments.add(segment); } } } @@ -304,10 +321,12 @@ public class HttpLoadQueuePeon extends LoadQueuePeon return; } + queuedSegments.remove(holder); + activeRequestSegments.remove(holder.getSegment()); if (status.getState() == SegmentLoadDropHandler.Status.STATE.FAILED) { - holder.requestFailed(status.getFailureCause()); + onRequestFailed(holder, status.getFailureCause()); } else { - holder.requestSucceeded(); + onRequestCompleted(holder, RequestStatus.SUCCESS); } } }, null @@ -347,44 +366,51 @@ public class HttpLoadQueuePeon extends LoadQueuePeon if (stopped) { return; } - + log.info("Stopping load queue peon for server [%s].", serverId); stopped = true; - for (SegmentHolder holder : segmentsToDrop.values()) { - holder.requestFailed("Stopping load queue peon."); - } - - for (SegmentHolder holder : segmentsToLoad.values()) { - holder.requestFailed("Stopping load queue peon."); - } + // Cancel all queued requests + queuedSegments.forEach(holder -> onRequestCompleted(holder, RequestStatus.CANCELLED)); + log.info("Cancelled [%d] requests queued on server [%s].", queuedSegments.size(), serverId); segmentsToDrop.clear(); segmentsToLoad.clear(); + queuedSegments.clear(); + activeRequestSegments.clear(); queuedSize.set(0L); - failedAssignCount.set(0); + stats.clear(); } } @Override - public void loadSegment(DataSegment segment, LoadPeonCallback callback) + public void loadSegment(DataSegment segment, SegmentAction action, LoadPeonCallback callback) { + if (!action.isLoad()) { + log.warn("Invalid load action [%s] for segment [%s] on server [%s].", action, segment.getId(), serverId); + return; + } + synchronized (lock) { if (stopped) { log.warn( "Server[%s] cannot load segment[%s] because load queue peon is stopped.", - serverId, - segment.getId() + serverId, segment.getId() ); - callback.execute(false); + if (callback != null) { + callback.execute(false); + } return; } SegmentHolder holder = segmentsToLoad.get(segment); - if (holder == null) { log.trace("Server[%s] to load segment[%s] queued.", serverId, segment.getId()); - segmentsToLoad.put(segment, new LoadSegmentHolder(segment, callback)); + queuedSize.addAndGet(segment.getSize()); + holder = new SegmentHolder(segment, action, callback); + segmentsToLoad.put(segment, holder); + queuedSegments.add(holder); processingExecutor.execute(this::doSegmentManagement); + incrementStat(holder, RequestStatus.ASSIGNED); } else { holder.addCallback(callback); } @@ -398,18 +424,22 @@ public class HttpLoadQueuePeon extends LoadQueuePeon if (stopped) { log.warn( "Server[%s] cannot drop segment[%s] because load queue peon is stopped.", - serverId, - segment.getId() + serverId, segment.getId() ); - callback.execute(false); + if (callback != null) { + callback.execute(false); + } return; } SegmentHolder holder = segmentsToDrop.get(segment); if (holder == null) { log.trace("Server[%s] to drop segment[%s] queued.", serverId, segment.getId()); - segmentsToDrop.put(segment, new DropSegmentHolder(segment, callback)); + holder = new SegmentHolder(segment, SegmentAction.DROP, callback); + segmentsToDrop.put(segment, holder); + queuedSegments.add(holder); processingExecutor.execute(this::doSegmentManagement); + incrementStat(holder, RequestStatus.ASSIGNED); } else { holder.addCallback(callback); } @@ -435,15 +465,25 @@ public class HttpLoadQueuePeon extends LoadQueuePeon } @Override - public long getLoadQueueSize() + public Set getSegmentsInQueue() + { + final Set segmentsInQueue; + synchronized (lock) { + segmentsInQueue = new HashSet<>(queuedSegments); + } + return segmentsInQueue; + } + + @Override + public long getSizeOfSegmentsToLoad() { return queuedSize.get(); } @Override - public int getAndResetFailedAssignCount() + public CoordinatorRunStats getAndResetStats() { - return failedAssignCount.getAndSet(0); + return stats.getSnapshotAndReset(); } @Override @@ -458,146 +498,101 @@ public class HttpLoadQueuePeon extends LoadQueuePeon segmentsMarkedToDrop.remove(dataSegment); } - @Override - public int getNumberOfSegmentsInQueue() - { - return segmentsToLoad.size(); - } - @Override public Set getSegmentsMarkedToDrop() { return Collections.unmodifiableSet(segmentsMarkedToDrop); } - private abstract class SegmentHolder + /** + * A request is considered to have timed out if the time elapsed since it was + * first sent to the server is greater than the configured load timeout. + * + * @see DruidCoordinatorConfig#getLoadTimeoutDelay() + */ + private boolean hasRequestTimedOut(SegmentHolder holder, long currentTimeMillis) { - private final DataSegment segment; - private final DataSegmentChangeRequest changeRequest; - private final List callbacks = new ArrayList<>(); + return holder.isRequestSentToServer() + && currentTimeMillis - holder.getFirstRequestMillis() + > config.getLoadTimeoutDelay().getMillis(); + } - // Time when this request was sent to target server the first time. - private volatile long scheduleTime = -1; + private void onRequestFailed(SegmentHolder holder, String failureCause) + { + log.error( + "Server[%s] failed segment[%s] request[%s] with cause [%s].", + serverId, holder.getSegment().getId(), holder.getAction(), failureCause + ); + onRequestCompleted(holder, RequestStatus.FAILED); + } - private SegmentHolder( - DataSegment segment, - DataSegmentChangeRequest changeRequest, - LoadPeonCallback callback - ) - { - this.segment = segment; - this.changeRequest = changeRequest; + private void onRequestCompleted(SegmentHolder holder, RequestStatus status) + { + final SegmentAction action = holder.getAction(); + log.trace( + "Server[%s] completed request[%s] on segment[%s] with status[%s].", + serverId, action, holder.getSegment().getId(), status + ); - if (callback != null) { - this.callbacks.add(callback); + if (holder.isLoad()) { + queuedSize.addAndGet(-holder.getSegment().getSize()); + } + incrementStat(holder, status); + executeCallbacks(holder, status == RequestStatus.SUCCESS); + } + + private void incrementStat(SegmentHolder holder, RequestStatus status) + { + stats.addToDatasourceStat(status.datasourceStat, holder.getSegment().getDataSource(), 1); + } + + private void executeCallbacks(SegmentHolder holder, boolean success) + { + callBackExecutor.execute(() -> { + for (LoadPeonCallback callback : holder.getCallbacks()) { + callback.execute(success); } - } + }); + } - public void addCallback(LoadPeonCallback newCallback) - { - synchronized (callbacks) { - if (newCallback != null) { - callbacks.add(newCallback); - } - } - } - - public DataSegment getSegment() - { - return segment; - } - - public DataSegmentChangeRequest getChangeRequest() - { - return changeRequest; - } - - public boolean hasTimedOut() - { - if (scheduleTime < 0) { - scheduleTime = System.currentTimeMillis(); - return false; - } else if (System.currentTimeMillis() - scheduleTime > config.getLoadTimeoutDelay().getMillis()) { - return true; - } else { + /** + * Tries to cancel a load/drop operation. An load/drop request can be cancelled + * only if it has not already been sent to the corresponding server. + */ + @Override + public boolean cancelOperation(DataSegment segment) + { + synchronized (lock) { + if (activeRequestSegments.contains(segment)) { return false; } - } - public void requestSucceeded() - { - log.trace( - "Server[%s] Successfully processed segment[%s] request[%s].", - serverId, - segment.getId(), - changeRequest.getClass().getSimpleName() - ); + // Find the action on this segment, if any + final SegmentHolder holder = segmentsToLoad.containsKey(segment) + ? segmentsToLoad.remove(segment) + : segmentsToDrop.remove(segment); + if (holder == null) { + return false; + } - callBackExecutor.execute(() -> { - for (LoadPeonCallback callback : callbacks) { - if (callback != null) { - callback.execute(true); - } - } - }); - } - - public void requestFailed(String failureCause) - { - log.error( - "Server[%s] Failed segment[%s] request[%s] with cause [%s].", - serverId, - segment.getId(), - changeRequest.getClass().getSimpleName(), - failureCause - ); - - failedAssignCount.getAndIncrement(); - - callBackExecutor.execute(() -> { - for (LoadPeonCallback callback : callbacks) { - if (callback != null) { - callback.execute(false); - } - } - }); - } - - @Override - public String toString() - { - return changeRequest.toString(); + queuedSegments.remove(holder); + onRequestCompleted(holder, RequestStatus.CANCELLED); + return true; } } - private class LoadSegmentHolder extends SegmentHolder + private enum RequestStatus { - public LoadSegmentHolder(DataSegment segment, LoadPeonCallback callback) - { - super(segment, new SegmentChangeRequestLoad(segment), callback); - queuedSize.addAndGet(segment.getSize()); - } + ASSIGNED(Stats.SegmentQueue.ASSIGNED_ACTIONS), + SUCCESS(Stats.SegmentQueue.COMPLETED_ACTIONS), + FAILED(Stats.SegmentQueue.FAILED_ACTIONS), + CANCELLED(Stats.SegmentQueue.CANCELLED_ACTIONS); - @Override - public void requestSucceeded() - { - queuedSize.addAndGet(-getSegment().getSize()); - super.requestSucceeded(); - } + final CoordinatorStat datasourceStat; - @Override - public void requestFailed(String failureCause) + RequestStatus(CoordinatorStat datasourceStat) { - queuedSize.addAndGet(-getSegment().getSize()); - super.requestFailed(failureCause); - } - } - - private class DropSegmentHolder extends SegmentHolder - { - public DropSegmentHolder(DataSegment segment, LoadPeonCallback callback) - { - super(segment, new SegmentChangeRequestDrop(segment), callback); + this.datasourceStat = datasourceStat; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/LoadPeonCallback.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java similarity index 96% rename from server/src/main/java/org/apache/druid/server/coordinator/LoadPeonCallback.java rename to server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java index d53d878e27f..2a2163563db 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/LoadPeonCallback.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadPeonCallback.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.loading; /** */ diff --git a/server/src/main/java/org/apache/druid/server/coordinator/LoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java similarity index 52% rename from server/src/main/java/org/apache/druid/server/coordinator/LoadQueuePeon.java rename to server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java index 483fe7f6f16..3ab6d30faf5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/LoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueuePeon.java @@ -17,8 +17,9 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.loading; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.timeline.DataSegment; import java.util.Set; @@ -28,30 +29,41 @@ import java.util.Set; * has been verified enough in production, CuratorLoadQueuePeon and this interface would be removed. */ @Deprecated -public abstract class LoadQueuePeon +public interface LoadQueuePeon { - public abstract void start(); - public abstract void stop(); + void start(); - public abstract Set getSegmentsToLoad(); + void stop(); - public abstract Set getSegmentsToDrop(); + Set getSegmentsToLoad(); - public abstract Set getTimedOutSegments(); + Set getSegmentsInQueue(); - public abstract void unmarkSegmentToDrop(DataSegment segmentToLoad); + Set getSegmentsToDrop(); + Set getTimedOutSegments(); - public abstract void markSegmentToDrop(DataSegment segmentToLoad); + void markSegmentToDrop(DataSegment segmentToLoad); - public abstract void loadSegment(DataSegment segment, LoadPeonCallback callback); - public abstract void dropSegment(DataSegment segment, LoadPeonCallback callback); + void unmarkSegmentToDrop(DataSegment segmentToLoad); - public abstract long getLoadQueueSize(); + Set getSegmentsMarkedToDrop(); - public abstract int getAndResetFailedAssignCount(); + void loadSegment(DataSegment segment, SegmentAction action, LoadPeonCallback callback); - public abstract int getNumberOfSegmentsInQueue(); - public abstract Set getSegmentsMarkedToDrop(); + void dropSegment(DataSegment segment, LoadPeonCallback callback); + + long getSizeOfSegmentsToLoad(); + + CoordinatorRunStats getAndResetStats(); + + /** + * Tries to cancel the current operation queued for the given segment on this + * server, if any. A request that has already been sent to the server cannot + * be cancelled. + * + * @return true if the operation was successfully cancelled + */ + boolean cancelOperation(DataSegment segment); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java similarity index 95% rename from server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java rename to server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java index 88c74dea0f8..bc68f87bc58 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/LoadQueueTaskMaster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.loading; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Provider; @@ -25,6 +25,7 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.utils.ZKPaths; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.initialization.ZkPathsConfig; import java.util.concurrent.ExecutorService; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/ReplicationThrottler.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/ReplicationThrottler.java new file mode 100644 index 00000000000..988a181707a --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/ReplicationThrottler.java @@ -0,0 +1,94 @@ +/* + * 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.druid.server.coordinator.loading; + +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * The ReplicationThrottler is used to throttle the number of segment replicas + * that are assigned to a load queue in a single run. This is achieved by setting + * the values of the following configs: + *
    + *
  • {@link CoordinatorDynamicConfig#getReplicationThrottleLimit()} - Maximum + * number of replicas that can be assigned to a tier in a single run.
  • + *
  • {@link CoordinatorDynamicConfig#getMaxNonPrimaryReplicantsToLoad()} - + * Maximum number of total replicas that can be assigned across all tiers in a + * single run.
  • + *
+ */ +public class ReplicationThrottler +{ + private final int replicationThrottleLimit; + private final int maxReplicaAssignmentsInRun; + + private final Map tierToNumAssigned = new HashMap<>(); + private final Set tiersLoadingReplicas = new HashSet<>(); + + private int totalReplicasAssignedInRun; + + /** + * Creates a new ReplicationThrottler for use during a single coordinator run. + * + * @param tiersLoadingReplicas Set of tier names which are already loading + * replicas and will not be eligible for loading + * more replicas in this run. + * @param replicationThrottleLimit Maximum number of replicas that can be + * assigned to a single tier in the current run. + * @param maxReplicaAssignmentsInRun Max number of total replicas that can be + * assigned across all tiers in the current run. + */ + public ReplicationThrottler( + Set tiersLoadingReplicas, + int replicationThrottleLimit, + int maxReplicaAssignmentsInRun + ) + { + this.replicationThrottleLimit = replicationThrottleLimit; + this.maxReplicaAssignmentsInRun = maxReplicaAssignmentsInRun; + this.totalReplicasAssignedInRun = 0; + if (tiersLoadingReplicas != null) { + this.tiersLoadingReplicas.addAll(tiersLoadingReplicas); + } + } + + public boolean isTierLoadingReplicas(String tier) + { + return tiersLoadingReplicas.contains(tier); + } + + public boolean canAssignReplica(String tier) + { + return !tiersLoadingReplicas.contains(tier) + && totalReplicasAssignedInRun < maxReplicaAssignmentsInRun + && tierToNumAssigned.computeIfAbsent(tier, t -> 0) < replicationThrottleLimit; + } + + public void incrementAssignedReplicas(String tier) + { + ++totalReplicasAssignedInRun; + tierToNumAssigned.compute(tier, (t, count) -> (count == null) ? 1 : count + 1); + } + +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/RoundRobinServerSelector.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/RoundRobinServerSelector.java similarity index 96% rename from server/src/main/java/org/apache/druid/server/coordinator/RoundRobinServerSelector.java rename to server/src/main/java/org/apache/druid/server/coordinator/loading/RoundRobinServerSelector.java index 89459adcc4f..1f9307cae56 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/RoundRobinServerSelector.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/RoundRobinServerSelector.java @@ -17,8 +17,10 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.loading; +import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; import javax.annotation.concurrent.NotThreadSafe; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentAction.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentAction.java new file mode 100644 index 00000000000..7c5e983afcc --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentAction.java @@ -0,0 +1,81 @@ +/* + * 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.druid.server.coordinator.loading; + +/** + * Represents actions that can be performed on a server for a single segment. + *

+ * The different action types can be used to prioritize items in a LoadQueuePeon. + */ +public enum SegmentAction +{ + /** + * Drop a segment from a server. + */ + DROP(false), + + /** + * Load a segment on a server. This should be used when trying to load a segment + * on a tier where it is currently unavailable (i.e. no replicas loaded). + * This action cannot be throttled by the {@code replicationThrottleLimit}. + */ + LOAD(true), + + /** + * Load a replica of a segment on a server. This should be used when trying to + * load more replicas of a segment on a tier where it is already available + * (i.e. atleast one loaded replica). + *

+ * This is different from LOAD in two ways: + *

    + *
  • this action can be throttled by the {@code replicationThrottleLimit}
  • + *
  • it is given lower priority than LOAD on the load queue peon
  • + *
+ * For all other purposes, REPLICATE is treated the same as LOAD. + */ + REPLICATE(true), + + /** + * Move a segment to this server. + */ + MOVE_TO(true), + + /** + * Move a segment from this server to another. This is essentially a pending + * DROP operation, which starts only when the corresponding MOVE_TO has succeded. + */ + MOVE_FROM(false); + + private final boolean isLoad; + + SegmentAction(boolean isLoad) + { + this.isLoad = isLoad; + } + + /** + * True only if this action loads a segment on a server, i.e. LOAD, REPLICATE + * or MOVE_TO. + */ + public boolean isLoad() + { + return isLoad; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java new file mode 100644 index 00000000000..ce199d65430 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentHolder.java @@ -0,0 +1,178 @@ +/* + * 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.druid.server.coordinator.loading; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Ordering; +import org.apache.druid.server.coordination.DataSegmentChangeRequest; +import org.apache.druid.server.coordination.SegmentChangeRequestDrop; +import org.apache.druid.server.coordination.SegmentChangeRequestLoad; +import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.timeline.DataSegment; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Represents a segment queued for a load or drop operation in a LoadQueuePeon. + *

+ * Requests are naturally ordered using the {@link #COMPARE_ACTION_THEN_INTERVAL}. + */ +public class SegmentHolder implements Comparable +{ + /** + * Orders segment requests: + *

    + *
  • first by action: all drops, then all loads, then all moves
  • + *
  • then by interval: newest segments first
  • + *
+ */ + private static final Comparator COMPARE_ACTION_THEN_INTERVAL = + Ordering.explicit(SegmentAction.DROP, SegmentAction.LOAD, SegmentAction.REPLICATE, SegmentAction.MOVE_TO) + .onResultOf(SegmentHolder::getAction) + .compound(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST.onResultOf(SegmentHolder::getSegment)); + + private final DataSegment segment; + private final DataSegmentChangeRequest changeRequest; + private final SegmentAction action; + + // Guaranteed to store only non-null elements + private final List callbacks = new ArrayList<>(); + private final AtomicLong firstRequestMillis = new AtomicLong(0); + private int runsInQueue = 0; + + public SegmentHolder( + DataSegment segment, + SegmentAction action, + @Nullable LoadPeonCallback callback + ) + { + this.segment = segment; + this.action = action; + this.changeRequest = (action == SegmentAction.DROP) + ? new SegmentChangeRequestDrop(segment) + : new SegmentChangeRequestLoad(segment); + if (callback != null) { + callbacks.add(callback); + } + } + + public DataSegment getSegment() + { + return segment; + } + + public SegmentAction getAction() + { + return action; + } + + public boolean isLoad() + { + return action != SegmentAction.DROP; + } + + public DataSegmentChangeRequest getChangeRequest() + { + return changeRequest; + } + + public String getSegmentIdentifier() + { + return segment.getId().toString(); + } + + public void addCallback(@Nullable LoadPeonCallback callback) + { + if (callback != null) { + synchronized (callbacks) { + callbacks.add(callback); + } + } + } + + /** + * Returns an immutable copy of all non-null callbacks for this queued segment. + */ + public List getCallbacks() + { + synchronized (callbacks) { + return ImmutableList.copyOf(callbacks); + } + } + + public void markRequestSentToServer() + { + firstRequestMillis.compareAndSet(0L, System.currentTimeMillis()); + } + + public boolean isRequestSentToServer() + { + return firstRequestMillis.get() > 0; + } + + public long getFirstRequestMillis() + { + return firstRequestMillis.get(); + } + + public int incrementAndGetRunsInQueue() + { + return ++runsInQueue; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SegmentHolder that = (SegmentHolder) o; + return getSegment().equals(that.getSegment()) && getAction() == that.getAction(); + } + + @Override + public int hashCode() + { + return Objects.hash(getSegment(), getAction()); + } + + @Override + public int compareTo(SegmentHolder that) + { + return Objects.compare(this, that, COMPARE_ACTION_THEN_INTERVAL); + } + + @Override + public String toString() + { + return action + "{" + + "segment=" + segment.getId() + + ", runsInQueue=" + runsInQueue + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java new file mode 100644 index 00000000000..dcaf87fb531 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java @@ -0,0 +1,159 @@ +/* + * 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.druid.server.coordinator.loading; + +import com.google.inject.Inject; +import org.apache.druid.client.ServerInventoryView; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.timeline.DataSegment; + +/** + * Manager for addition/removal of segments to server load queues and the + * corresponding success/failure callbacks. + */ +public class SegmentLoadQueueManager +{ + private static final Logger log = new Logger(SegmentLoadQueueManager.class); + + private final LoadQueueTaskMaster taskMaster; + private final ServerInventoryView serverInventoryView; + private final SegmentsMetadataManager segmentsMetadataManager; + + @Inject + public SegmentLoadQueueManager( + ServerInventoryView serverInventoryView, + SegmentsMetadataManager segmentsMetadataManager, + LoadQueueTaskMaster taskMaster + ) + { + this.serverInventoryView = serverInventoryView; + this.segmentsMetadataManager = segmentsMetadataManager; + this.taskMaster = taskMaster; + } + + /** + * Queues load of the segment on the given server. + */ + public boolean loadSegment(DataSegment segment, ServerHolder server, SegmentAction action) + { + try { + if (!server.startOperation(action, segment)) { + return false; + } + + server.getPeon().loadSegment(segment, action, null); + return true; + } + catch (Exception e) { + server.cancelOperation(action, segment); + final String serverName = server.getServer().getName(); + log.error(e, "Error while loading segment[%s] on server[%s]", segment.getId(), serverName); + return false; + } + } + + public boolean dropSegment(DataSegment segment, ServerHolder server) + { + try { + if (server.startOperation(SegmentAction.DROP, segment)) { + server.getPeon().dropSegment(segment, null); + return true; + } else { + return false; + } + } + catch (Exception e) { + server.cancelOperation(SegmentAction.DROP, segment); + final String serverName = server.getServer().getName(); + log.error(e, "Error while dropping segment[%s] from server[%s]", segment.getId(), serverName); + return false; + } + } + + public boolean moveSegment( + DataSegment segment, + ServerHolder serverA, + ServerHolder serverB + ) + { + final LoadQueuePeon peonA = serverA.getPeon(); + final LoadPeonCallback moveFinishCallback = success -> peonA.unmarkSegmentToDrop(segment); + + if (!serverA.startOperation(SegmentAction.MOVE_FROM, segment)) { + return false; + } + if (!serverB.startOperation(SegmentAction.MOVE_TO, segment)) { + serverA.cancelOperation(SegmentAction.MOVE_FROM, segment); + return false; + } + + // mark segment to drop before it is actually loaded on server + // to be able to account for this information in BalancerStrategy immediately + peonA.markSegmentToDrop(segment); + + final LoadQueuePeon peonB = serverB.getPeon(); + final String serverNameB = serverB.getServer().getName(); + try { + peonB.loadSegment( + segment, + SegmentAction.MOVE_TO, + success -> { + // Drop segment only if: + // (1) segment load was successful on serverB + // AND (2) segment is not already queued for drop on serverA + // AND (3a) loading is http-based + // OR (3b) inventory shows segment loaded on serverB + + // Do not check the inventory with http loading as the HTTP + // response is enough to determine load success or failure + if (success + && !peonA.getSegmentsToDrop().contains(segment) + && (taskMaster.isHttpLoading() + || serverInventoryView.isSegmentLoadedByServer(serverNameB, segment))) { + peonA.unmarkSegmentToDrop(segment); + peonA.dropSegment(segment, moveFinishCallback); + } else { + moveFinishCallback.execute(success); + } + } + ); + } + catch (Exception e) { + serverA.cancelOperation(SegmentAction.MOVE_FROM, segment); + serverB.cancelOperation(SegmentAction.MOVE_TO, segment); + moveFinishCallback.execute(false); + log.error(e, "Error while moving segment[%s] to server[%s]", segment.getId(), serverNameB); + return false; + } + + return true; + } + + /** + * Marks the given segment as unused. + */ + public boolean deleteSegment(DataSegment segment) + { + return segmentsMetadataManager.markSegmentAsUnused(segment.getId()); + } + +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java new file mode 100644 index 00000000000..559950c18e7 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.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.druid.server.coordinator.loading; + +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; + +/** + * Contains recomputed configs from {@link CoordinatorDynamicConfig} based on + * whether {@link CoordinatorDynamicConfig#isSmartSegmentLoading} is enabled or not. + */ +public class SegmentLoadingConfig +{ + private static final Logger log = new Logger(SegmentLoadingConfig.class); + + private final int maxSegmentsInLoadQueue; + private final int replicationThrottleLimit; + private final int maxReplicaAssignmentsInRun; + private final int maxLifetimeInLoadQueue; + + private final int maxSegmentsToMove; + private final int percentDecommSegmentsToMove; + + private final boolean useRoundRobinSegmentAssignment; + private final boolean emitBalancingStats; + + /** + * Creates a new SegmentLoadingConfig with recomputed coordinator config values from + * based on whether {@link CoordinatorDynamicConfig#isSmartSegmentLoading()} + * is enabled or not. + */ + public static SegmentLoadingConfig create(CoordinatorDynamicConfig dynamicConfig, int numUsedSegments) + { + if (dynamicConfig.isSmartSegmentLoading()) { + // Compute recommended values + // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove + final int throttlePercentage = 2; + final int replicationThrottleLimit = Math.max(100, numUsedSegments * throttlePercentage / 100); + + // Impose an upper bound on maxSegmentsToMove to ensure that coordinator + // run times are bounded. This limit can be relaxed as performance of + // the CostBalancerStrategy.computeCost() is improved. + final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit); + + log.info( + "Smart segment loading is enabled. Recomputed replicationThrottleLimit" + + " [%,d] (%d%% of used segments [%,d]) and maxSegmentsToMove [%d].", + replicationThrottleLimit, throttlePercentage, numUsedSegments, maxSegmentsToMove + ); + + return new SegmentLoadingConfig( + 0, + replicationThrottleLimit, + Integer.MAX_VALUE, + dynamicConfig.getReplicantLifetime(), + maxSegmentsToMove, + 100, + true, + false + ); + } else { + // Use the configured values + return new SegmentLoadingConfig( + dynamicConfig.getMaxSegmentsInNodeLoadingQueue(), + dynamicConfig.getReplicationThrottleLimit(), + dynamicConfig.getMaxNonPrimaryReplicantsToLoad(), + dynamicConfig.getReplicantLifetime(), + dynamicConfig.getMaxSegmentsToMove(), + dynamicConfig.getDecommissioningMaxPercentOfMaxSegmentsToMove(), + dynamicConfig.isUseRoundRobinSegmentAssignment(), + dynamicConfig.emitBalancingStats() + ); + } + } + + private SegmentLoadingConfig( + int maxSegmentsInLoadQueue, + int replicationThrottleLimit, + int maxReplicaAssignmentsInRun, + int maxLifetimeInLoadQueue, + int maxSegmentsToMove, + int percentDecommSegmentsToMove, + boolean useRoundRobinSegmentAssignment, + boolean emitBalancingStats + ) + { + this.maxSegmentsInLoadQueue = maxSegmentsInLoadQueue; + this.replicationThrottleLimit = replicationThrottleLimit; + this.maxReplicaAssignmentsInRun = maxReplicaAssignmentsInRun; + this.maxLifetimeInLoadQueue = maxLifetimeInLoadQueue; + this.maxSegmentsToMove = maxSegmentsToMove; + this.percentDecommSegmentsToMove = percentDecommSegmentsToMove; + this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment; + this.emitBalancingStats = emitBalancingStats; + } + + public int getMaxSegmentsInLoadQueue() + { + return maxSegmentsInLoadQueue; + } + + public int getMaxSegmentsToMove() + { + return maxSegmentsToMove; + } + + public int getReplicationThrottleLimit() + { + return replicationThrottleLimit; + } + + public boolean isUseRoundRobinSegmentAssignment() + { + return useRoundRobinSegmentAssignment; + } + + public boolean isEmitBalancingStats() + { + return emitBalancingStats; + } + + public int getMaxLifetimeInLoadQueue() + { + return maxLifetimeInLoadQueue; + } + + public int getMaxReplicaAssignmentsInRun() + { + return maxReplicaAssignmentsInRun; + } + + public int getPercentDecommSegmentsToMove() + { + return percentDecommSegmentsToMove; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicaCount.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicaCount.java new file mode 100644 index 00000000000..bcaa23083a6 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicaCount.java @@ -0,0 +1,167 @@ +/* + * 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.druid.server.coordinator.loading; + +/** + * Counts the number of replicas of a segment in different states (loading, loaded, etc) + * in a tier or the whole cluster. + */ +public class SegmentReplicaCount +{ + private int requiredAndLoadable; + private int required; + + private int loaded; + private int loadedNonHistorical; + + private int loading; + private int dropping; + private int moving; + + /** + * Increments number of replicas loaded on historical servers. + */ + void incrementLoaded() + { + ++loaded; + } + + /** + * Increments number of replicas loaded on non-historical servers. This value + * is used only for computing level of under-replication of broadcast segments. + */ + void incrementLoadedOnNonHistoricalServer() + { + ++loadedNonHistorical; + } + + /** + * Increments number of replicas queued for the given action. + */ + void incrementQueued(SegmentAction action) + { + switch (action) { + case REPLICATE: + case LOAD: + ++loading; + break; + case MOVE_TO: + ++moving; + break; + case DROP: + ++dropping; + break; + default: + break; + } + } + + /** + * Sets the number of required replicas of this segment. + * + * @param required Number of replicas as required by load or broadcast rules. + * @param numLoadingServers Number of servers that can load replicas of this segment. + */ + void setRequired(int required, int numLoadingServers) + { + this.required = required; + this.requiredAndLoadable = Math.min(required, numLoadingServers); + } + + /** + * Required number of replicas of the segment as dictated by load rules. + * This includes replicas that may be in excess of the cluster capacity. + */ + public int required() + { + return required; + } + + /** + * Required number of replicas of the segment as dictated by load rules. + * This does not include replicas that are in excess of the cluster capacity. + */ + public int requiredAndLoadable() + { + return requiredAndLoadable; + } + + int loading() + { + return loading; + } + + int moving() + { + return moving; + } + + /** + * Number of replicas loaded on all servers. This includes replicas that are + * currently being dropped. + */ + public int totalLoaded() + { + return loaded + loadedNonHistorical; + } + + /** + * Number of replicas which are safely loaded on historical servers and are + * not being dropped. + */ + int loadedNotDropping() + { + return loaded - dropping; + } + + /** + * Number of replicas that are required to be loaded but are missing. + * This includes replicas that may be in excess of the cluster capacity. + */ + int missing() + { + return Math.max(required() - totalLoaded(), 0); + } + + /** + * Number of replicas that are required to be loaded but are missing. + * This does not include replicas that are in excess of the cluster capacity. + */ + int missingAndLoadable() + { + return Math.max(requiredAndLoadable() - totalLoaded(), 0); + } + + /** + * Accumulates counts from the given {@code SegmentReplicaCount} into this instance. + */ + void accumulate(SegmentReplicaCount other) + { + this.required += other.required; + this.requiredAndLoadable += other.requiredAndLoadable; + + this.loaded += other.loaded; + this.loadedNonHistorical += other.loadedNonHistorical; + + this.loading += other.loading; + this.dropping += other.dropping; + this.moving += other.moving; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicaCountMap.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicaCountMap.java new file mode 100644 index 00000000000..3d3e34072fb --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicaCountMap.java @@ -0,0 +1,108 @@ +/* + * 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.druid.server.coordinator.loading; + +import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * Contains a mapping from tier to {@link SegmentReplicaCount}s. + *

+ * Used by the {@link StrategicSegmentAssigner} to make assignment decisions. + */ +public class SegmentReplicaCountMap +{ + private final Map> replicaCounts = new HashMap<>(); + + static SegmentReplicaCountMap create(DruidCluster cluster) + { + final SegmentReplicaCountMap replicaCountMap = new SegmentReplicaCountMap(); + replicaCountMap.initReplicaCounts(cluster); + return replicaCountMap; + } + + private void initReplicaCounts(DruidCluster cluster) + { + cluster.getHistoricals().forEach( + (tier, historicals) -> historicals.forEach( + serverHolder -> { + // Add segments already loaded on this server + for (DataSegment segment : serverHolder.getServedSegments()) { + computeIfAbsent(segment.getId(), tier).incrementLoaded(); + } + + // Add segments queued for load, drop or move on this server + serverHolder.getQueuedSegments().forEach( + (segment, state) -> computeIfAbsent(segment.getId(), tier) + .incrementQueued(state) + ); + } + ) + ); + + cluster.getBrokers().forEach(broker -> { + final ImmutableDruidServer server = broker.getServer(); + for (DataSegment segment : server.iterateAllSegments()) { + computeIfAbsent(segment.getId(), server.getTier()) + .incrementLoadedOnNonHistoricalServer(); + } + }); + + cluster.getRealtimes().forEach(realtime -> { + final ImmutableDruidServer server = realtime.getServer(); + for (DataSegment segment : server.iterateAllSegments()) { + computeIfAbsent(segment.getId(), server.getTier()) + .incrementLoadedOnNonHistoricalServer(); + } + }); + } + + SegmentReplicaCount get(SegmentId segmentId, String tier) + { + SegmentReplicaCount count = replicaCounts.getOrDefault(segmentId, Collections.emptyMap()) + .get(tier); + return count == null ? new SegmentReplicaCount() : count; + } + + SegmentReplicaCount getTotal(SegmentId segmentId) + { + final SegmentReplicaCount total = new SegmentReplicaCount(); + replicaCounts.getOrDefault(segmentId, Collections.emptyMap()) + .values().forEach(total::accumulate); + return total; + } + + public SegmentReplicaCount computeIfAbsent(SegmentId segmentId, String tier) + { + return replicaCounts.computeIfAbsent(segmentId, s -> new HashMap<>()) + .computeIfAbsent(tier, t -> new SegmentReplicaCount()); + } + + public SegmentReplicationStatus toReplicationStatus() + { + return new SegmentReplicationStatus(replicaCounts); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicationStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicationStatus.java new file mode 100644 index 00000000000..7121642f25e --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicationStatus.java @@ -0,0 +1,84 @@ +/* + * 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.druid.server.coordinator.loading; + +import com.google.common.collect.ImmutableMap; +import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; + +import java.util.HashMap; +import java.util.Map; + +/** + * An immutable object that contains information about the under-replicated + * or unavailable status of all used segments. This state is recomputed by + * the {@link StrategicSegmentAssigner} in every run. + */ +public class SegmentReplicationStatus +{ + private final Map totalReplicaCounts; + private final Map> replicaCountsInTier; + + public SegmentReplicationStatus(Map> replicaCountsInTier) + { + this.replicaCountsInTier = ImmutableMap.copyOf(replicaCountsInTier); + + final Map totalReplicaCounts = new HashMap<>(); + replicaCountsInTier.forEach((segmentId, tierToReplicaCount) -> { + final SegmentReplicaCount total = new SegmentReplicaCount(); + tierToReplicaCount.values().forEach(total::accumulate); + totalReplicaCounts.put(segmentId, total); + }); + this.totalReplicaCounts = ImmutableMap.copyOf(totalReplicaCounts); + } + + public SegmentReplicaCount getReplicaCountsInCluster(SegmentId segmentId) + { + return totalReplicaCounts.get(segmentId); + } + + public Map> getTierToDatasourceToUnderReplicated( + Iterable usedSegments, + boolean ignoreMissingServers + ) + { + final Map> tierToUnderReplicated = new HashMap<>(); + + for (DataSegment segment : usedSegments) { + final Map tierToReplicaCount = replicaCountsInTier.get(segment.getId()); + if (tierToReplicaCount == null) { + continue; + } + + tierToReplicaCount.forEach((tier, counts) -> { + final int underReplicated = ignoreMissingServers ? counts.missing() : counts.missingAndLoadable(); + if (underReplicated >= 0) { + Object2LongOpenHashMap datasourceToUnderReplicated = (Object2LongOpenHashMap) + tierToUnderReplicated.computeIfAbsent(tier, ds -> new Object2LongOpenHashMap<>()); + datasourceToUnderReplicated.addTo(segment.getDataSource(), underReplicated); + } + }); + } + + return tierToUnderReplicated; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentStatusInTier.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentStatusInTier.java new file mode 100644 index 00000000000..e9977829f5d --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentStatusInTier.java @@ -0,0 +1,78 @@ +/* + * 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.druid.server.coordinator.loading; + +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.timeline.DataSegment; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; + +/** + * Maintains a map containing the state of a segment on all servers of a tier. + */ +public class SegmentStatusInTier +{ + private final DataSegment segment; + + private final List eligibleLoadServers = new ArrayList<>(); + private final List eligibleDropServers = new ArrayList<>(); + + private final Map> serversWithQueuedActions = new HashMap<>(); + + public SegmentStatusInTier(DataSegment segment, NavigableSet historicals) + { + this.segment = segment; + historicals.forEach(this::handleServer); + } + + public List getServersEligibleToLoad() + { + return eligibleLoadServers; + } + + public List getServersEligibleToDrop() + { + return eligibleDropServers; + } + + public List getServersPerforming(SegmentAction action) + { + return serversWithQueuedActions.getOrDefault(action, Collections.emptyList()); + } + + private void handleServer(ServerHolder server) + { + final SegmentAction action = server.getActionOnSegment(segment); + if (server.isServingSegment(segment)) { + eligibleDropServers.add(server); + } else if (server.canLoadSegment(segment)) { + eligibleLoadServers.add(server); + } else if (action != null) { + serversWithQueuedActions.computeIfAbsent(action, a -> new ArrayList<>()) + .add(server); + } + } + +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java new file mode 100644 index 00000000000..4fc2e176f75 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java @@ -0,0 +1,606 @@ +/* + * 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.druid.server.coordinator.loading; + +import com.google.common.collect.Sets; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.balancer.BalancerStrategy; +import org.apache.druid.server.coordinator.rules.SegmentActionHandler; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.CoordinatorStat; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; + +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; + +/** + * Used by the coordinator in each run for segment loading, dropping, balancing + * and broadcasting. + *

+ * An instance of this class is freshly created for each coordinator run. + */ +public class StrategicSegmentAssigner implements SegmentActionHandler +{ + private static final EmittingLogger log = new EmittingLogger(StrategicSegmentAssigner.class); + + private final SegmentLoadQueueManager loadQueueManager; + private final DruidCluster cluster; + private final CoordinatorRunStats stats; + private final SegmentReplicaCountMap replicaCountMap; + private final ReplicationThrottler replicationThrottler; + private final RoundRobinServerSelector serverSelector; + private final BalancerStrategy strategy; + + private final boolean useRoundRobinAssignment; + + private final Set tiersWithNoServer = new HashSet<>(); + private final Map tierToHistoricalCount = new HashMap<>(); + + public StrategicSegmentAssigner( + SegmentLoadQueueManager loadQueueManager, + DruidCluster cluster, + BalancerStrategy strategy, + SegmentLoadingConfig loadingConfig, + CoordinatorRunStats stats + ) + { + this.stats = stats; + this.cluster = cluster; + this.strategy = strategy; + this.loadQueueManager = loadQueueManager; + this.replicaCountMap = SegmentReplicaCountMap.create(cluster); + this.replicationThrottler = createReplicationThrottler(cluster, loadingConfig); + this.useRoundRobinAssignment = loadingConfig.isUseRoundRobinSegmentAssignment(); + this.serverSelector = useRoundRobinAssignment ? new RoundRobinServerSelector(cluster) : null; + + cluster.getHistoricals().forEach( + (tier, historicals) -> tierToHistoricalCount.put(tier, historicals.size()) + ); + } + + public CoordinatorRunStats getStats() + { + return stats; + } + + public SegmentReplicationStatus getReplicationStatus() + { + return replicaCountMap.toReplicationStatus(); + } + + public void makeAlerts() + { + if (!tiersWithNoServer.isEmpty()) { + log.makeAlert("Tiers [%s] have no servers! Check your cluster configuration.", tiersWithNoServer).emit(); + } + } + + /** + * Moves the given segment from the source server to an eligible destination + * server. + *

+ * An eligible destination server must: + *

    + *
  • be present in the given list of destination servers
  • + *
  • belong to the same tier as the source server
  • + *
  • not already be serving or loading a replica of the segment
  • + *
  • have enough space to load the segment
  • + *
+ *

+ * The segment is not moved if: + *

    + *
  • there is no eligible destination server
  • + *
  • or segment is already optimally placed
  • + *
  • or some other error occurs
  • + *
+ */ + public boolean moveSegment( + DataSegment segment, + ServerHolder sourceServer, + List destinationServers + ) + { + final String tier = sourceServer.getServer().getTier(); + final List eligibleDestinationServers = + destinationServers.stream() + .filter(s -> s.getServer().getTier().equals(tier)) + .filter(s -> s.canLoadSegment(segment)) + .collect(Collectors.toList()); + + if (eligibleDestinationServers.isEmpty()) { + incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "No eligible server", segment, tier); + return false; + } + + // If the source server is not decommissioning, move can be skipped if the + // segment is already optimally placed + if (!sourceServer.isDecommissioning()) { + eligibleDestinationServers.add(sourceServer); + } + + final ServerHolder destination = + strategy.findDestinationServerToMoveSegment(segment, sourceServer, eligibleDestinationServers); + + if (destination == null || destination.getServer().equals(sourceServer.getServer())) { + incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Optimally placed", segment, tier); + return false; + } else if (moveSegment(segment, sourceServer, destination)) { + incrementStat(Stats.Segments.MOVED, segment, tier, 1); + return true; + } else { + incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Encountered error", segment, tier); + return false; + } + } + + /** + * Moves the given segment from serverA to serverB. + */ + private boolean moveSegment(DataSegment segment, ServerHolder serverA, ServerHolder serverB) + { + final String tier = serverA.getServer().getTier(); + if (serverA.isLoadingSegment(segment)) { + // Cancel the load on serverA and load on serverB instead + if (serverA.cancelOperation(SegmentAction.LOAD, segment)) { + int loadedCountOnTier = replicaCountMap.get(segment.getId(), tier) + .loadedNotDropping(); + if (loadedCountOnTier >= 1) { + return replicateSegment(segment, serverB); + } else { + return loadSegment(segment, serverB); + } + } + + // Could not cancel load, let the segment load on serverA and count it as unmoved + return false; + } else if (serverA.isServingSegment(segment)) { + return loadQueueManager.moveSegment(segment, serverA, serverB); + } else { + return false; + } + } + + @Override + public void replicateSegment(DataSegment segment, Map tierToReplicaCount) + { + // Identify empty tiers and determine total required replicas + final Set allTiersInCluster = Sets.newHashSet(cluster.getTierNames()); + tierToReplicaCount.forEach((tier, requiredReplicas) -> { + reportTierCapacityStats(segment, requiredReplicas, tier); + + SegmentReplicaCount replicaCount = replicaCountMap.computeIfAbsent(segment.getId(), tier); + replicaCount.setRequired(requiredReplicas, tierToHistoricalCount.getOrDefault(tier, 0)); + + if (!allTiersInCluster.contains(tier)) { + tiersWithNoServer.add(tier); + } + }); + + SegmentReplicaCount replicaCountInCluster = replicaCountMap.getTotal(segment.getId()); + final int replicaSurplus = replicaCountInCluster.loadedNotDropping() + - replicaCountInCluster.requiredAndLoadable(); + + // Update replicas in every tier + int dropsQueued = 0; + for (String tier : allTiersInCluster) { + dropsQueued += updateReplicasInTier( + segment, + tier, + tierToReplicaCount.getOrDefault(tier, 0), + replicaSurplus - dropsQueued + ); + } + } + + /** + * Queues load or drop operations on this tier based on the required + * number of replicas and the current state. + *

+ * The {@code maxReplicasToDrop} helps to maintain the required level of + * replication in the cluster. This ensures that segment read concurrency does + * not suffer during a tier shift or load rule change. + *

+ * Returns the number of new drop operations queued on this tier. + */ + private int updateReplicasInTier( + DataSegment segment, + String tier, + int requiredReplicas, + int maxReplicasToDrop + ) + { + final SegmentReplicaCount replicaCountOnTier + = replicaCountMap.get(segment.getId(), tier); + + final int projectedReplicas = replicaCountOnTier.loadedNotDropping() + + replicaCountOnTier.loading(); + + final int movingReplicas = replicaCountOnTier.moving(); + final boolean shouldCancelMoves = requiredReplicas == 0 && movingReplicas > 0; + + // Check if there is any action required on this tier + if (projectedReplicas == requiredReplicas && !shouldCancelMoves) { + return 0; + } + + final SegmentStatusInTier segmentStatus = + new SegmentStatusInTier(segment, cluster.getHistoricalsByTier(tier)); + + // Cancel all moves in this tier if it does not need to have replicas + if (shouldCancelMoves) { + cancelOperations(SegmentAction.MOVE_TO, movingReplicas, segment, segmentStatus); + cancelOperations(SegmentAction.MOVE_FROM, movingReplicas, segment, segmentStatus); + } + + // Cancel drops and queue loads if the projected count is below the requirement + if (projectedReplicas < requiredReplicas) { + int replicaDeficit = requiredReplicas - projectedReplicas; + int cancelledDrops = + cancelOperations(SegmentAction.DROP, replicaDeficit, segment, segmentStatus); + + // Cancelled drops can be counted as loaded replicas, thus reducing deficit + int numReplicasToLoad = replicaDeficit - cancelledDrops; + if (numReplicasToLoad > 0) { + int numLoadedReplicas = replicaCountOnTier.loadedNotDropping() + cancelledDrops; + int numLoadsQueued = loadReplicas(numReplicasToLoad, numLoadedReplicas, segment, tier, segmentStatus); + incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued); + } + } + + // Cancel loads and queue drops if the projected count exceeds the requirement + if (projectedReplicas > requiredReplicas) { + int replicaSurplus = projectedReplicas - requiredReplicas; + int cancelledLoads = + cancelOperations(SegmentAction.LOAD, replicaSurplus, segment, segmentStatus); + + int numReplicasToDrop = Math.min(replicaSurplus - cancelledLoads, maxReplicasToDrop); + if (numReplicasToDrop > 0) { + int dropsQueuedOnTier = dropReplicas(numReplicasToDrop, segment, tier, segmentStatus); + incrementStat(Stats.Segments.DROPPED, segment, tier, dropsQueuedOnTier); + return dropsQueuedOnTier; + } + } + + return 0; + } + + private void reportTierCapacityStats(DataSegment segment, int requiredReplicas, String tier) + { + final RowKey rowKey = RowKey.forTier(tier); + stats.updateMax(Stats.Tier.REPLICATION_FACTOR, rowKey, requiredReplicas); + stats.add(Stats.Tier.REQUIRED_CAPACITY, rowKey, segment.getSize() * requiredReplicas); + } + + @Override + public void broadcastSegment(DataSegment segment) + { + final Object2IntOpenHashMap tierToRequiredReplicas = new Object2IntOpenHashMap<>(); + for (ServerHolder server : cluster.getAllServers()) { + // Ignore servers which are not broadcast targets + if (!server.getServer().getType().isSegmentBroadcastTarget()) { + continue; + } + + final String tier = server.getServer().getTier(); + + // Drop from decommissioning servers and load on active servers + int numDropsQueued = 0; + int numLoadsQueued = 0; + if (server.isDecommissioning()) { + numDropsQueued += dropBroadcastSegment(segment, server) ? 1 : 0; + } else { + tierToRequiredReplicas.addTo(tier, 1); + numLoadsQueued += loadBroadcastSegment(segment, server) ? 1 : 0; + } + + if (numLoadsQueued > 0) { + incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued); + } + if (numDropsQueued > 0) { + incrementStat(Stats.Segments.DROPPED, segment, tier, numDropsQueued); + } + } + + // Update required replica counts + tierToRequiredReplicas.object2IntEntrySet().fastForEach( + entry -> replicaCountMap.computeIfAbsent(segment.getId(), entry.getKey()) + .setRequired(entry.getIntValue(), entry.getIntValue()) + ); + } + + @Override + public void deleteSegment(DataSegment segment) + { + loadQueueManager.deleteSegment(segment); + stats.addToDatasourceStat(Stats.Segments.DELETED, segment.getDataSource(), 1); + } + + /** + * Loads the broadcast segment if it is not loaded on the given server. + * Returns true only if the segment was successfully queued for load on the server. + */ + private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server) + { + if (server.isServingSegment(segment) || server.isLoadingSegment(segment)) { + return false; + } else if (server.isDroppingSegment(segment)) { + return server.cancelOperation(SegmentAction.DROP, segment); + } + + if (server.canLoadSegment(segment) && loadSegment(segment, server)) { + return true; + } else { + log.makeAlert("Could not assign broadcast segment for datasource [%s]", segment.getDataSource()) + .addData("segmentId", segment.getId()) + .addData("segmentSize", segment.getSize()) + .addData("hostName", server.getServer().getHost()) + .addData("availableSize", server.getAvailableSize()) + .emit(); + return false; + } + } + + /** + * Drops the broadcast segment if it is loaded on the given server. + * Returns true only if the segment was successfully queued for drop on the server. + */ + private boolean dropBroadcastSegment(DataSegment segment, ServerHolder server) + { + if (server.isLoadingSegment(segment)) { + return server.cancelOperation(SegmentAction.LOAD, segment); + } else if (server.isServingSegment(segment)) { + return loadQueueManager.dropSegment(segment, server); + } else { + return false; + } + } + + /** + * Queues drop of {@code numToDrop} replicas of the segment from a tier. + * Tries to drop replicas first from decommissioning servers and then from + * active servers. + *

+ * Returns the number of successfully queued drop operations. + */ + private int dropReplicas( + final int numToDrop, + DataSegment segment, + String tier, + SegmentStatusInTier segmentStatus + ) + { + if (numToDrop <= 0) { + return 0; + } + + final List eligibleServers = segmentStatus.getServersEligibleToDrop(); + if (eligibleServers.isEmpty()) { + incrementSkipStat(Stats.Segments.DROP_SKIPPED, "No eligible server", segment, tier); + return 0; + } + + // Keep eligible servers sorted by most full first + final TreeSet eligibleLiveServers = new TreeSet<>(Comparator.reverseOrder()); + final TreeSet eligibleDyingServers = new TreeSet<>(Comparator.reverseOrder()); + for (ServerHolder server : eligibleServers) { + if (server.isDecommissioning()) { + eligibleDyingServers.add(server); + } else { + eligibleLiveServers.add(server); + } + } + + // Drop as many replicas as possible from decommissioning servers + int remainingNumToDrop = numToDrop; + int numDropsQueued = + dropReplicasFromServers(remainingNumToDrop, segment, eligibleDyingServers.iterator(), tier); + + // Drop replicas from active servers if required + if (numToDrop > numDropsQueued) { + remainingNumToDrop = numToDrop - numDropsQueued; + Iterator serverIterator = + (useRoundRobinAssignment || eligibleLiveServers.size() >= remainingNumToDrop) + ? eligibleLiveServers.iterator() + : strategy.pickServersToDropSegment(segment, eligibleLiveServers); + numDropsQueued += dropReplicasFromServers(remainingNumToDrop, segment, serverIterator, tier); + } + + return numDropsQueued; + } + + /** + * Queues drop of {@code numToDrop} replicas of the segment from the servers. + * Returns the number of successfully queued drop operations. + */ + private int dropReplicasFromServers( + int numToDrop, + DataSegment segment, + Iterator serverIterator, + String tier + ) + { + int numDropsQueued = 0; + while (numToDrop > numDropsQueued && serverIterator.hasNext()) { + ServerHolder holder = serverIterator.next(); + boolean dropped = loadQueueManager.dropSegment(segment, holder); + + if (dropped) { + ++numDropsQueued; + } else { + incrementSkipStat(Stats.Segments.DROP_SKIPPED, "Encountered error", segment, tier); + } + } + + return numDropsQueued; + } + + /** + * Queues load of {@code numToLoad} replicas of the segment on a tier. + */ + private int loadReplicas( + int numToLoad, + int numLoadedReplicas, + DataSegment segment, + String tier, + SegmentStatusInTier segmentStatus + ) + { + final boolean isAlreadyLoadedOnTier = numLoadedReplicas >= 1; + + // Do not assign replicas if tier is already busy loading some + if (isAlreadyLoadedOnTier && replicationThrottler.isTierLoadingReplicas(tier)) { + return 0; + } + + final List eligibleServers = segmentStatus.getServersEligibleToLoad(); + if (eligibleServers.isEmpty()) { + incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No eligible server", segment, tier); + return 0; + } + + final Iterator serverIterator = + useRoundRobinAssignment + ? serverSelector.getServersInTierToLoadSegment(tier, segment) + : strategy.findServersToLoadSegment(segment, eligibleServers); + if (!serverIterator.hasNext()) { + incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No server chosen by strategy", segment, tier); + return 0; + } + + // Load the replicas on this tier + int numLoadsQueued = 0; + while (numLoadsQueued < numToLoad && serverIterator.hasNext()) { + ServerHolder server = serverIterator.next(); + boolean queuedSuccessfully = isAlreadyLoadedOnTier ? replicateSegment(segment, server) + : loadSegment(segment, server); + numLoadsQueued += queuedSuccessfully ? 1 : 0; + } + + return numLoadsQueued; + } + + private boolean loadSegment(DataSegment segment, ServerHolder server) + { + final String tier = server.getServer().getTier(); + final boolean assigned = loadQueueManager.loadSegment(segment, server, SegmentAction.LOAD); + + if (!assigned) { + incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Encountered error", segment, tier); + } + + return assigned; + } + + private boolean replicateSegment(DataSegment segment, ServerHolder server) + { + final String tier = server.getServer().getTier(); + if (!replicationThrottler.canAssignReplica(tier)) { + incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Throttled replication", segment, tier); + return false; + } + + final boolean assigned = loadQueueManager.loadSegment(segment, server, SegmentAction.REPLICATE); + if (!assigned) { + incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Encountered error", segment, tier); + } else { + replicationThrottler.incrementAssignedReplicas(tier); + } + + return assigned; + } + + private static ReplicationThrottler createReplicationThrottler( + DruidCluster cluster, + SegmentLoadingConfig loadingConfig + ) + { + final Set tiersLoadingReplicas = new HashSet<>(); + + cluster.getHistoricals().forEach( + (tier, historicals) -> { + int numLoadingReplicas = historicals.stream().mapToInt(ServerHolder::getNumLoadingReplicas).sum(); + if (numLoadingReplicas > 0) { + log.info( + "Tier [%s] will not be assigned replicas as it is already loading [%d] replicas.", + tier, numLoadingReplicas + ); + tiersLoadingReplicas.add(tier); + } + } + ); + return new ReplicationThrottler( + tiersLoadingReplicas, + loadingConfig.getReplicationThrottleLimit(), + loadingConfig.getMaxReplicaAssignmentsInRun() + ); + } + + private int cancelOperations( + SegmentAction action, + int maxNumToCancel, + DataSegment segment, + SegmentStatusInTier segmentStatus + ) + { + final List servers = segmentStatus.getServersPerforming(action); + if (servers.isEmpty() || maxNumToCancel <= 0) { + return 0; + } + + int numCancelled = 0; + for (int i = 0; i < servers.size() && numCancelled < maxNumToCancel; ++i) { + numCancelled += servers.get(i).cancelOperation(action, segment) ? 1 : 0; + } + return numCancelled; + } + + private void incrementSkipStat(CoordinatorStat stat, String reason, DataSegment segment, String tier) + { + final RowKey.Builder keyBuilder + = RowKey.builder() + .add(Dimension.TIER, tier) + .add(Dimension.DATASOURCE, segment.getDataSource()); + + if (reason != null) { + keyBuilder.add(Dimension.DESCRIPTION, reason); + } + + stats.add(stat, keyBuilder.build(), 1); + } + + private void incrementStat(CoordinatorStat stat, DataSegment segment, String tier, long value) + { + stats.addToSegmentStat(stat, tier, segment.getDataSource(), value); + } + +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java index 58b2ecb4bd2..5fdcec4e043 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java @@ -19,152 +19,15 @@ package org.apache.druid.server.coordinator.rules; -import it.unimi.dsi.fastutil.objects.Object2LongMap; -import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidCluster; -import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.SegmentReplicantLookup; -import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - public abstract class BroadcastDistributionRule implements Rule { - private static final EmittingLogger log = new EmittingLogger(BroadcastDistributionRule.class); @Override - public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment) + public void run(DataSegment segment, SegmentActionHandler handler) { - final Set dropServerHolders = new HashSet<>(); - - // Find servers where we need to load the broadcast segments - final Set loadServerHolders = - params.getDruidCluster().getAllServers() - .stream() - .filter( - (serverHolder) -> { - ServerType serverType = serverHolder.getServer().getType(); - if (!serverType.isSegmentBroadcastTarget()) { - return false; - } - - final boolean isServingSegment = - serverHolder.isServingSegment(segment); - - if (serverHolder.isDecommissioning()) { - if (isServingSegment && !serverHolder.isDroppingSegment(segment)) { - dropServerHolders.add(serverHolder); - } - return false; - } - - return !isServingSegment && !serverHolder.isLoadingSegment(segment); - } - ) - .collect(Collectors.toSet()); - - final CoordinatorStats stats = new CoordinatorStats(); - return stats.accumulate(assign(loadServerHolders, segment)) - .accumulate(drop(dropServerHolders, segment)); + handler.broadcastSegment(segment); } - @Override - public boolean canLoadSegments() - { - return true; - } - - @Override - public void updateUnderReplicated( - Map> underReplicatedPerTier, - SegmentReplicantLookup segmentReplicantLookup, - DataSegment segment - ) - { - Object2LongMap underReplicatedBroadcastTiers = segmentReplicantLookup.getBroadcastUnderReplication(segment.getId()); - for (final Object2LongMap.Entry entry : underReplicatedBroadcastTiers.object2LongEntrySet()) { - final String tier = entry.getKey(); - final long underReplicatedCount = entry.getLongValue(); - underReplicatedPerTier.compute(tier, (_tier, existing) -> { - Object2LongMap underReplicationPerDataSource = existing; - if (existing == null) { - underReplicationPerDataSource = new Object2LongOpenHashMap<>(); - } - underReplicationPerDataSource.compute( - segment.getDataSource(), - (_datasource, count) -> count != null ? count + underReplicatedCount : underReplicatedCount - ); - return underReplicationPerDataSource; - }); - } - } - - @Override - public void updateUnderReplicatedWithClusterView( - Map> underReplicatedPerTier, - SegmentReplicantLookup segmentReplicantLookup, - DruidCluster cluster, - DataSegment segment - ) - { - updateUnderReplicated( - underReplicatedPerTier, - segmentReplicantLookup, - segment - ); - } - - private CoordinatorStats assign( - final Set serverHolders, - final DataSegment segment - ) - { - final CoordinatorStats stats = new CoordinatorStats(); - stats.addToGlobalStat(LoadRule.ASSIGNED_COUNT, 0); - - for (ServerHolder holder : serverHolders) { - if (segment.getSize() > holder.getAvailableSize()) { - log.makeAlert("Failed to broadcast segment for [%s]", segment.getDataSource()) - .addData("segmentId", segment.getId()) - .addData("segmentSize", segment.getSize()) - .addData("hostName", holder.getServer().getHost()) - .addData("availableSize", holder.getAvailableSize()) - .emit(); - } else { - if (!holder.isLoadingSegment(segment)) { - holder.getPeon().loadSegment( - segment, - null - ); - - stats.addToGlobalStat(LoadRule.ASSIGNED_COUNT, 1); - } - } - } - - return stats; - } - - private CoordinatorStats drop( - final Set serverHolders, - final DataSegment segment - ) - { - CoordinatorStats stats = new CoordinatorStats(); - - for (ServerHolder holder : serverHolders) { - holder.getPeon().dropSegment(segment, null); - stats.addToGlobalStat(LoadRule.DROPPED_COUNT, 1); - } - - return stats; - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java index 7ffc7a21155..59b9db9d506 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java @@ -19,9 +19,6 @@ package org.apache.druid.server.coordinator.rules; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.timeline.DataSegment; /** @@ -30,17 +27,9 @@ import org.apache.druid.timeline.DataSegment; public abstract class DropRule implements Rule { @Override - public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment) + public void run(DataSegment segment, SegmentActionHandler handler) { - CoordinatorStats stats = new CoordinatorStats(); - coordinator.markSegmentAsUnused(segment); - stats.addToGlobalStat("deletedCount", 1); - return stats; + handler.deleteSegment(segment); } - @Override - public boolean canLoadSegments() - { - return false; - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverLoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverLoadRule.java index ee5da0bdbb2..dc1798f13e0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverLoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverLoadRule.java @@ -41,7 +41,9 @@ public class ForeverLoadRule extends LoadRule @JsonProperty("tieredReplicants") Map tieredReplicants ) { - this.tieredReplicants = tieredReplicants == null ? ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) : tieredReplicants; + this.tieredReplicants = tieredReplicants == null + ? ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) + : tieredReplicants; validateTieredReplicants(this.tieredReplicants); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java index c89a74edbff..548c25cad54 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java @@ -19,535 +19,20 @@ package org.apache.druid.server.coordinator.rules; -import it.unimi.dsi.fastutil.objects.Object2IntMap; -import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; -import it.unimi.dsi.fastutil.objects.Object2LongMap; -import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.coordinator.BalancerStrategy; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidCluster; -import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.ReplicationThrottler; -import org.apache.druid.server.coordinator.SegmentReplicantLookup; -import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; import java.util.Map; -import java.util.NavigableSet; -import java.util.TreeSet; -import java.util.function.Predicate; -import java.util.stream.Collectors; /** * LoadRules indicate the number of replicants a segment should have in a given tier. */ public abstract class LoadRule implements Rule { - private static final EmittingLogger log = new EmittingLogger(LoadRule.class); - static final String ASSIGNED_COUNT = "assignedCount"; - static final String DROPPED_COUNT = "droppedCount"; - public final String NON_PRIMARY_ASSIGNED_COUNT = "totalNonPrimaryReplicantsLoaded"; - public static final String REQUIRED_CAPACITY = "requiredCapacity"; - - private final Object2IntMap targetReplicants = new Object2IntOpenHashMap<>(); - private final Object2IntMap currentReplicants = new Object2IntOpenHashMap<>(); - - // Cache to hold unused results from strategy call in assignPrimary - private final Map strategyCache = new HashMap<>(); - @Override - public CoordinatorStats run( - final DruidCoordinator coordinator, - final DruidCoordinatorRuntimeParams params, - final DataSegment segment - ) + public void run(DataSegment segment, SegmentActionHandler handler) { - try { - // get the "snapshots" of targetReplicants and currentReplicants for assignments. - targetReplicants.putAll(getTieredReplicants()); - currentReplicants.putAll(params.getSegmentReplicantLookup().getClusterTiers(segment.getId())); - - params.getSegmentReplicantLookup().setReplicationFactor(segment.getId(), getReplicationFactor()); - - final CoordinatorStats stats = new CoordinatorStats(); - assign(params, segment, stats); - - drop(params, segment, stats); - for (String tier : targetReplicants.keySet()) { - stats.addToTieredStat(REQUIRED_CAPACITY, tier, segment.getSize() * targetReplicants.getInt(tier)); - } - return stats; - } - finally { - targetReplicants.clear(); - currentReplicants.clear(); - strategyCache.clear(); - } - } - - private int getReplicationFactor() - { - return getTieredReplicants().values().stream().reduce(0, Integer::sum); - } - - @Override - public boolean canLoadSegments() - { - return true; - } - - @Override - public void updateUnderReplicated( - Map> underReplicatedPerTier, - SegmentReplicantLookup segmentReplicantLookup, - DataSegment segment - ) - { - getTieredReplicants().forEach((final String tier, final Integer ruleReplicants) -> { - int currentReplicants = segmentReplicantLookup.getLoadedReplicants(segment.getId(), tier); - Object2LongMap underReplicationPerDataSource = underReplicatedPerTier.computeIfAbsent( - tier, - ignored -> new Object2LongOpenHashMap<>() - ); - ((Object2LongOpenHashMap) underReplicationPerDataSource).addTo( - segment.getDataSource(), - Math.max(ruleReplicants - currentReplicants, 0) - ); - }); - } - - @Override - public void updateUnderReplicatedWithClusterView( - Map> underReplicatedPerTier, - SegmentReplicantLookup segmentReplicantLookup, - DruidCluster cluster, - DataSegment segment - ) - { - getTieredReplicants().forEach((final String tier, final Integer ruleReplicants) -> { - int currentReplicants = segmentReplicantLookup.getLoadedReplicants(segment.getId(), tier); - Object2LongMap underReplicationPerDataSource = underReplicatedPerTier.computeIfAbsent( - tier, - ignored -> new Object2LongOpenHashMap<>() - ); - int possibleReplicants = Math.min(ruleReplicants, cluster.getHistoricals().get(tier).size()); - log.debug( - "ruleReplicants: [%d], possibleReplicants: [%d], currentReplicants: [%d]", - ruleReplicants, - possibleReplicants, - currentReplicants - ); - ((Object2LongOpenHashMap) underReplicationPerDataSource).addTo( - segment.getDataSource(), - Math.max(possibleReplicants - currentReplicants, 0) - ); - }); - } - - /** - * @param stats {@link CoordinatorStats} to accumulate assignment statistics. - */ - private void assign( - final DruidCoordinatorRuntimeParams params, - final DataSegment segment, - final CoordinatorStats stats - ) - { - // if primary replica already exists or is loading - final int loading = params.getSegmentReplicantLookup().getTotalReplicants(segment.getId()); - if (!currentReplicants.isEmpty() || loading > 0) { - assignReplicas(params, segment, stats, null); - } else { - final ServerHolder primaryHolderToLoad = assignPrimary(params, segment); - if (primaryHolderToLoad == null) { - // cluster does not have any replicants and cannot identify primary holder - // this implies that no assignment could be done - return; - } - - int numAssigned = 1; // 1 replica (i.e., primary replica) already assigned - - final String tier = primaryHolderToLoad.getServer().getTier(); - // assign replicas for the rest of the tier - numAssigned += assignReplicasForTier( - tier, - targetReplicants.getOrDefault(tier, 0), - numAssigned, // note that the currentReplicantsInTier is the just-assigned primary replica. - params, - createLoadQueueSizeLimitingPredicate(segment).and(holder -> !holder.equals(primaryHolderToLoad)), - segment - ); - - // numAssigned - 1 because we don't want to count the primary assignment - stats.addToGlobalStat(NON_PRIMARY_ASSIGNED_COUNT, numAssigned - 1); - - stats.addToTieredStat(ASSIGNED_COUNT, tier, numAssigned); - - // do assign replicas for the other tiers. - assignReplicas(params, segment, stats, tier /* to skip */); - } - } - - private static Predicate createLoadQueueSizeLimitingPredicate( - final DataSegment segment - ) - { - return server -> server != null && server.canLoadSegment(segment); - } - - private static List getFilteredHolders( - final String tier, - final DruidCluster druidCluster, - final Predicate predicate - ) - { - final NavigableSet queue = druidCluster.getHistoricalsByTier(tier); - if (queue == null) { - log.makeAlert("Tier[%s] has no servers! Check your cluster configuration!", tier).emit(); - return Collections.emptyList(); - } - Predicate isActive = s -> !s.isDecommissioning(); - return queue.stream().filter(isActive.and(predicate)).collect(Collectors.toList()); - } - - private Iterator getRoundRobinIterator( - DruidCoordinatorRuntimeParams params, - String tier, - DataSegment segment - ) - { - if (params.getRoundRobinServerSelector() == null - || !params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) { - return null; - } - - return params.getRoundRobinServerSelector() - .getServersInTierToLoadSegment(tier, segment); - } - - /** - * Iterates through each tier and find the respective segment homes; with the found segment homes, selects the one - * with the highest priority to be the holder for the primary replica. - */ - @Nullable - private ServerHolder assignPrimary( - final DruidCoordinatorRuntimeParams params, - final DataSegment segment - ) - { - ServerHolder topCandidate = null; - final boolean useRoundRobinAssignment = params.getCoordinatorDynamicConfig() - .isUseRoundRobinSegmentAssignment(); - for (final Object2IntMap.Entry entry : targetReplicants.object2IntEntrySet()) { - final int targetReplicantsInTier = entry.getIntValue(); - // sanity check: target number of replicants should be more than zero. - if (targetReplicantsInTier <= 0) { - continue; - } - final String tier = entry.getKey(); - - String noAvailability = StringUtils.format( - "No available [%s] servers or node capacity to assign primary segment [%s]! %s", - tier, - segment.getId(), - getReplicationLogString() - ); - - final List holders = getFilteredHolders( - tier, - params.getDruidCluster(), - createLoadQueueSizeLimitingPredicate(segment) - ); - // no holders satisfy the predicate - if (holders.isEmpty()) { - log.warn(noAvailability); - continue; - } - - final ServerHolder candidate; - if (useRoundRobinAssignment) { - Iterator roundRobinIterator = getRoundRobinIterator(params, tier, segment); - candidate = roundRobinIterator.hasNext() ? roundRobinIterator.next() : null; - } else { - candidate = params.getBalancerStrategy().findNewSegmentHomeReplicator(segment, holders); - if (candidate != null) { - strategyCache.put(tier, candidate); - } - } - - if (candidate == null) { - log.warn(noAvailability); - } else { - if (topCandidate == null || - candidate.getServer().getPriority() > topCandidate.getServer().getPriority()) { - topCandidate = candidate; - } - } - } - - if (topCandidate != null) { - // remove tier for primary replica - strategyCache.remove(topCandidate.getServer().getTier()); - log.info( - "Assigning 'primary' for segment [%s] to server [%s] in tier [%s]", - segment.getId(), - topCandidate.getServer().getName(), - topCandidate.getServer().getTier() - ); - topCandidate.getPeon().loadSegment(segment, null); - } - - return topCandidate; - } - - /** - * @param stats {@link CoordinatorStats} to accumulate assignment statistics. - * @param tierToSkip if not null, this tier will be skipped from doing assignment, use when primary replica was - * assigned. - */ - private void assignReplicas( - final DruidCoordinatorRuntimeParams params, - final DataSegment segment, - final CoordinatorStats stats, - @Nullable final String tierToSkip - ) - { - for (final Object2IntMap.Entry entry : targetReplicants.object2IntEntrySet()) { - final String tier = entry.getKey(); - if (tier.equals(tierToSkip)) { - log.info("Skipping replica assignment for segment [%s] to tier [%s]", segment.getId(), tier); - continue; - } - final int numAssigned = assignReplicasForTier( - tier, - entry.getIntValue(), - params.getSegmentReplicantLookup().getTotalReplicants(segment.getId(), tier), - params, - createLoadQueueSizeLimitingPredicate(segment), - segment - ); - stats.addToGlobalStat(NON_PRIMARY_ASSIGNED_COUNT, numAssigned); - stats.addToTieredStat(ASSIGNED_COUNT, tier, numAssigned); - } - } - - /** - * @param predicate {@link Predicate} used to pre-filter {@link ServerHolder}s retrieved from {@link DruidCluster}. - */ - private int assignReplicasForTier( - final String tier, - final int targetReplicantsInTier, - final int currentReplicantsInTier, - final DruidCoordinatorRuntimeParams params, - final Predicate predicate, - final DataSegment segment - ) - { - final int numToAssign = targetReplicantsInTier - currentReplicantsInTier; - // if nothing to assign - if (numToAssign <= 0) { - return 0; - } - - String noAvailability = StringUtils.format( - "No available [%s] servers or node capacity to assign segment [%s]! %s", - tier, - segment.getId(), - getReplicationLogString() - ); - - final List holders = getFilteredHolders(tier, params.getDruidCluster(), predicate); - // if no holders available for assignment - if (holders.isEmpty()) { - log.warn(noAvailability); - return 0; - } - - final Iterator roundRobinServerIterator = getRoundRobinIterator(params, tier, segment); - final ReplicationThrottler throttler = params.getReplicationManager(); - for (int numAssigned = 0; numAssigned < numToAssign; numAssigned++) { - if (!throttler.canCreateReplicant(tier)) { - log.info("Throttling replication for segment [%s] in tier [%s]. %s", segment.getId(), tier, getReplicationLogString()); - return numAssigned; - } - - // Retrieves from cache if available - final ServerHolder holder; - if (strategyCache.containsKey(tier)) { - // found in cache - holder = strategyCache.remove(tier); - } else if (roundRobinServerIterator == null) { - // Call balancer strategy - holder = params.getBalancerStrategy().findNewSegmentHomeReplicator(segment, holders); - } else { - holder = roundRobinServerIterator.hasNext() ? roundRobinServerIterator.next() : null; - } - - if (holder == null) { - log.warn(noAvailability); - return numAssigned; - } - holders.remove(holder); - - final SegmentId segmentId = segment.getId(); - final String holderHost = holder.getServer().getHost(); - throttler.registerReplicantCreation(tier, segmentId, holderHost); - log.info( - "Assigning 'replica' for segment [%s] to server [%s] in tier [%s]. %s", - segment.getId(), - holder.getServer().getName(), - holder.getServer().getTier(), - getReplicationLogString() - ); - holder.getPeon().loadSegment( - segment, - loadSuccess -> throttler.unregisterReplicantCreation(tier, segmentId) - ); - } - - return numToAssign; - } - - /** - * @param stats {@link CoordinatorStats} to accumulate assignment statistics. - */ - private void drop( - final DruidCoordinatorRuntimeParams params, - final DataSegment segment, - final CoordinatorStats stats - ) - { - final DruidCluster druidCluster = params.getDruidCluster(); - - - final boolean isLoading = loadingInProgress(druidCluster); - - for (final Object2IntMap.Entry entry : currentReplicants.object2IntEntrySet()) { - final String tier = entry.getKey(); - - final NavigableSet holders = druidCluster.getHistoricalsByTier(tier); - - final int numDropped; - if (holders == null) { - log.makeAlert("No holders found for tier[%s]", tier).emit(); - numDropped = 0; - } else { - final int currentReplicantsInTier = entry.getIntValue(); - final int numToDrop = currentReplicantsInTier - targetReplicants.getOrDefault(tier, 0); - if (numToDrop > 0) { - // This enforces that loading is completed before we attempt to drop stuffs as a safety measure. - if (isLoading) { - log.info( - "Loading in progress for segment [%s], skipping drop from tier [%s] until loading is complete! %s", - segment.getId(), - tier, - getReplicationLogString() - ); - break; - } - numDropped = dropForTier( - numToDrop, - holders, - segment, - params.getBalancerStrategy(), - getReplicationLogString() - ); - } else { - numDropped = 0; - } - } - - stats.addToTieredStat(DROPPED_COUNT, tier, numDropped); - } - } - - /** - * Returns true if at least one tier in target replica assignment exists in cluster but does not have enough replicas. - */ - private boolean loadingInProgress(final DruidCluster druidCluster) - { - for (final Object2IntMap.Entry entry : targetReplicants.object2IntEntrySet()) { - final String tier = entry.getKey(); - // if there are replicants loading in cluster - if (druidCluster.hasTier(tier) && entry.getIntValue() > currentReplicants.getOrDefault(tier, 0)) { - return true; - } - } - - return false; - } - - private static int dropForTier( - final int numToDrop, - final NavigableSet holdersInTier, - final DataSegment segment, - final BalancerStrategy balancerStrategy, - final String replicationLog - ) - { - Map> holders = holdersInTier.stream() - .filter(s -> s.isServingSegment(segment)) - .collect(Collectors.partitioningBy( - ServerHolder::isDecommissioning, - Collectors.toCollection(TreeSet::new) - )); - TreeSet decommissioningServers = holders.get(true); - TreeSet activeServers = holders.get(false); - int left = dropSegmentFromServers(balancerStrategy, segment, decommissioningServers, numToDrop, replicationLog); - if (left > 0) { - left = dropSegmentFromServers(balancerStrategy, segment, activeServers, left, replicationLog); - } - if (left != 0) { - log.warn("I have no servers serving [%s]?", segment.getId()); - } - return numToDrop - left; - } - - private static int dropSegmentFromServers( - final BalancerStrategy balancerStrategy, - final DataSegment segment, - final NavigableSet holders, - int numToDrop, - final String replicationLog - ) - { - final Iterator iterator = balancerStrategy.pickServersToDrop(segment, holders); - - while (numToDrop > 0) { - if (!iterator.hasNext()) { - break; - } - - final ServerHolder holder = iterator.next(); - if (holder.isServingSegment(segment)) { - log.info( - "Dropping segment [%s] on server [%s] in tier [%s]. %s", - segment.getId(), - holder.getServer().getName(), - holder.getServer().getTier(), - replicationLog - ); - holder.getPeon().dropSegment(segment, null); - numToDrop--; - } else { - log.warn( - "Server [%s] is no longer serving segment [%s], skipping drop.", - holder.getServer().getName(), - segment.getId() - ); - } - } - return numToDrop; + handler.replicateSegment(segment, getTieredReplicants()); } protected static void validateTieredReplicants(final Map tieredReplicants) @@ -569,20 +54,4 @@ public abstract class LoadRule implements Rule public abstract int getNumReplicants(String tier); - protected String getReplicationLogString() - { - StringBuilder builder = new StringBuilder("Current replication: ["); - for (final Object2IntMap.Entry entry : currentReplicants.object2IntEntrySet()) { - final String tier = entry.getKey(); - // [hot:1/2][cold:2/2] - builder.append("[") - .append(tier) - .append(":") - .append(entry.getIntValue()) - .append("/") - .append(targetReplicants.getInt(tier)) - .append("]"); - } - return builder.append("]").toString(); - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java index 1674dac3529..a66101d0fe1 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java @@ -21,20 +21,12 @@ package org.apache.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.base.Preconditions; -import it.unimi.dsi.fastutil.objects.Object2LongMap; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidCluster; -import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.SegmentReplicantLookup; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.Map; - /** + * Retention rule that governs retention and distribution of segments in a cluster. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @@ -57,54 +49,5 @@ public interface Rule boolean appliesTo(Interval interval, DateTime referenceTimestamp); - /** - * Return true if this Rule can load segment onto one or more type of Druid node, otherwise return false. - * Any Rule that returns true for this method should implement logic for calculating segment under replicated - * in {@link Rule#updateUnderReplicated} - */ - boolean canLoadSegments(); - - /** - * This method should update the {@param underReplicatedPerTier} with the replication count of the - * {@param segment}. Rule that returns true for {@link Rule#canLoadSegments()} must override this method. - * Note that {@param underReplicatedPerTier} is a map of tier -> { dataSource -> underReplicationCount } - */ - default void updateUnderReplicated( - Map> underReplicatedPerTier, - SegmentReplicantLookup segmentReplicantLookup, - DataSegment segment - ) - { - Preconditions.checkArgument(!canLoadSegments()); - } - - /** - * This method should update the {@param underReplicatedPerTier} with the replication count of the - * {@param segment} taking into consideration the number of servers available in cluster that the segment can be - * replicated on. Rule that returns true for {@link Rule#canLoadSegments()} must override this method. - * Note that {@param underReplicatedPerTier} is a map of tier -> { dataSource -> underReplicationCount } - */ - default void updateUnderReplicatedWithClusterView( - Map> underReplicatedPerTier, - SegmentReplicantLookup segmentReplicantLookup, - DruidCluster cluster, - DataSegment segment - ) - { - Preconditions.checkArgument(!canLoadSegments()); - } - - /** - * {@link DruidCoordinatorRuntimeParams#getUsedSegments()} must not be called in Rule's code, because the used - * segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is because - * {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions: - * "CoordinatorDutyParams" and "RuleParams" which contain params that only {@link - * org.apache.druid.server.coordinator.duty.CoordinatorDuty} objects and Rules need, respectively. For example, - * {@link org.apache.druid.server.coordinator.ReplicationThrottler} needs to belong only to "RuleParams", but not to - * "CoordinatorDutyParams". The opposite for the collection of used segments and {@link - * org.apache.druid.client.DataSourcesSnapshot}. - * - * See https://github.com/apache/druid/issues/7228 - */ - CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment); + void run(DataSegment segment, SegmentActionHandler segmentHandler); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/RuleMap.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/SegmentActionHandler.java similarity index 53% rename from server/src/main/java/org/apache/druid/server/coordinator/rules/RuleMap.java rename to server/src/main/java/org/apache/druid/server/coordinator/rules/SegmentActionHandler.java index 0ecaafe88e9..dc76735c135 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/RuleMap.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/SegmentActionHandler.java @@ -19,32 +19,32 @@ package org.apache.druid.server.coordinator.rules; -import java.util.ArrayList; -import java.util.List; +import org.apache.druid.timeline.DataSegment; + import java.util.Map; /** + * Performs various actions on a given segment. Used by {@link Rule}s to load, + * drop, broadcast or delete segments. */ -public class RuleMap +public interface SegmentActionHandler { - private final Map> rules; - private final List defaultRules; - public RuleMap(Map> rules, List defaultRules) - { - this.rules = rules; - this.defaultRules = defaultRules; - } + /** + * Queues load or drop of replicas of the given segment to achieve the + * target replication level on all historical tiers. + */ + void replicateSegment(DataSegment segment, Map tierToReplicaCount); + + /** + * Marks the given segment as unused. Unused segments are eventually unloaded + * from all servers and deleted from metadata as well as deep storage. + */ + void deleteSegment(DataSegment segment); + + /** + * Broadcasts the given segment to all servers that are broadcast targets. + */ + void broadcastSegment(DataSegment segment); - public List getRules(String dataSource) - { - List retVal = new ArrayList<>(); - if (dataSource != null) { - retVal.addAll(rules.get(dataSource)); - } - if (defaultRules != null) { - retVal.addAll(defaultRules); - } - return retVal; - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java new file mode 100644 index 00000000000..7d63c78c479 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java @@ -0,0 +1,273 @@ +/* + * 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.druid.server.coordinator.stats; + +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; +import org.apache.druid.java.util.common.StringUtils; + +import javax.annotation.concurrent.ThreadSafe; +import java.util.Collections; +import java.util.EnumMap; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Contains statistics typically tracked during a single coordinator run or the + * runtime of a single coordinator duty. + */ +@ThreadSafe +public class CoordinatorRunStats +{ + private final ConcurrentHashMap> + allStats = new ConcurrentHashMap<>(); + private final Map debugDimensions = new HashMap<>(); + + public CoordinatorRunStats() + { + this(null); + } + + /** + * Creates a new {@code CoordinatorRunStats}. + * + * @param debugDimensions Dimension values for which all metrics should be + * collected and logged. + */ + public CoordinatorRunStats(Map debugDimensions) + { + if (debugDimensions != null) { + this.debugDimensions.putAll(debugDimensions); + } + } + + public long getSegmentStat(CoordinatorStat stat, String tier, String datasource) + { + return get(stat, RowKey.builder().add(Dimension.DATASOURCE, datasource).add(Dimension.TIER, tier).build()); + } + + public long getDataSourceStat(CoordinatorStat stat, String dataSource) + { + return get(stat, RowKey.forDatasource(dataSource)); + } + + public long get(CoordinatorStat stat) + { + return get(stat, RowKey.EMPTY); + } + + public long get(CoordinatorStat stat, RowKey rowKey) + { + Object2LongOpenHashMap statValues = allStats.get(rowKey); + return statValues == null ? 0 : statValues.getLong(stat); + } + + public void forEachStat(StatHandler handler) + { + allStats.forEach( + (rowKey, stats) -> stats.object2LongEntrySet().fastForEach( + stat -> handler.handle(rowKey.getValues(), stat.getKey(), stat.getLongValue()) + ) + ); + } + + /** + * Builds a printable table of all the collected error, info and debug level + * stats (if applicable) with non-zero values. + */ + public String buildStatsTable() + { + final StringBuilder statsTable = new StringBuilder(); + final AtomicInteger hiddenStats = new AtomicInteger(0); + final AtomicInteger totalStats = new AtomicInteger(); + + allStats.forEach( + (rowKey, statMap) -> { + // Categorize the stats by level + final Map> levelToStats + = new EnumMap<>(CoordinatorStat.Level.class); + + statMap.object2LongEntrySet().fastForEach( + stat -> levelToStats.computeIfAbsent(stat.getKey().getLevel(), l -> new HashMap<>()) + .put(stat.getKey(), stat.getLongValue()) + ); + + // Add all the errors + final Map errorStats = levelToStats + .getOrDefault(CoordinatorStat.Level.ERROR, Collections.emptyMap()); + totalStats.addAndGet(errorStats.size()); + if (!errorStats.isEmpty()) { + statsTable.append( + StringUtils.format("\nError: %s ==> %s", rowKey, errorStats) + ); + } + + // Add all the info level stats + final Map infoStats = levelToStats + .getOrDefault(CoordinatorStat.Level.INFO, Collections.emptyMap()); + totalStats.addAndGet(infoStats.size()); + if (!infoStats.isEmpty()) { + statsTable.append( + StringUtils.format("\nInfo : %s ==> %s", rowKey, infoStats) + ); + } + + // Add all the debug level stats if the row key has a debug dimension + final Map debugStats = levelToStats + .getOrDefault(CoordinatorStat.Level.DEBUG, Collections.emptyMap()); + totalStats.addAndGet(debugStats.size()); + if (!debugStats.isEmpty() && hasDebugDimension(rowKey)) { + statsTable.append( + StringUtils.format("\nDebug: %s ==> %s", rowKey, debugStats) + ); + } else { + hiddenStats.addAndGet(debugStats.size()); + } + } + ); + + if (hiddenStats.get() > 0) { + statsTable.append( + StringUtils.format("\nDebug: %d hidden stats. Set 'debugDimensions' to see these.", hiddenStats.get()) + ); + } + if (totalStats.get() > 0) { + statsTable.append( + StringUtils.format("\nTOTAL: %d stats for %d dimension keys", totalStats.get(), rowCount()) + ); + } + + return statsTable.toString(); + } + + public boolean hasStat(CoordinatorStat stat) + { + for (Object2LongOpenHashMap statValues : allStats.values()) { + if (statValues.containsKey(stat)) { + return true; + } + } + return false; + } + + public int rowCount() + { + return allStats.size(); + } + + public void clear() + { + allStats.clear(); + } + + public void add(CoordinatorStat stat, long value) + { + add(stat, RowKey.EMPTY, value); + } + + public void add(CoordinatorStat stat, RowKey rowKey, long value) + { + // Do not add a stat which will neither be emitted nor logged + if (!stat.shouldEmit() + && stat.getLevel() == CoordinatorStat.Level.DEBUG + && !hasDebugDimension(rowKey)) { + return; + } + + allStats.computeIfAbsent(rowKey, d -> new Object2LongOpenHashMap<>()) + .addTo(stat, value); + } + + public void addToDatasourceStat(CoordinatorStat stat, String dataSource, long value) + { + add(stat, RowKey.forDatasource(dataSource), value); + } + + public void addToSegmentStat(CoordinatorStat stat, String tier, String datasource, long value) + { + RowKey rowKey = RowKey.builder() + .add(Dimension.TIER, tier) + .add(Dimension.DATASOURCE, datasource).build(); + add(stat, rowKey, value); + } + + /** + * Updates the maximum value of the stat for the given RowKey if applicable. + */ + public void updateMax(CoordinatorStat stat, RowKey rowKey, long value) + { + allStats.computeIfAbsent(rowKey, d -> new Object2LongOpenHashMap<>()) + .mergeLong(stat, value, Math::max); + } + + /** + * Creates a new {@code CoordinatorRunStats} which represents the snapshot of + * the stats collected so far in this instance. + *

+ * While this method is in progress, any updates made to the stats of this + * instance by another thread are not guaranteed to be present in the snapshot. + * But the snapshots are consistent, i.e. stats present in the snapshot created + * in one invocation of this method are permanently removed from this instance + * and will not be present in subsequent snapshots. + * + * @return Snapshot of the current state of this {@code CoordinatorRunStats}. + */ + public CoordinatorRunStats getSnapshotAndReset() + { + final CoordinatorRunStats snapshot = new CoordinatorRunStats(debugDimensions); + + // Get a snapshot of all the keys, remove and copy each of them atomically + final Set keys = new HashSet<>(allStats.keySet()); + for (RowKey key : keys) { + snapshot.allStats.put(key, allStats.remove(key)); + } + + return snapshot; + } + + /** + * Checks if the given rowKey has any of the debug dimensions. + */ + private boolean hasDebugDimension(RowKey rowKey) + { + if (debugDimensions.isEmpty()) { + return false; + } + + for (Map.Entry entry : rowKey.getValues().entrySet()) { + String expectedValue = debugDimensions.get(entry.getKey()); + if (Objects.equals(expectedValue, entry.getValue())) { + return true; + } + } + + return false; + } + + public interface StatHandler + { + void handle(Map dimensionValues, CoordinatorStat stat, long statValue); + } + +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorStat.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorStat.java new file mode 100644 index 00000000000..a5d53f3735f --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorStat.java @@ -0,0 +1,99 @@ +/* + * 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.druid.server.coordinator.stats; + +/** + * A coordinator statistic, which may or may not be emitted as a metric. + */ +public class CoordinatorStat +{ + private final String metricName; + private final String shortName; + private final Level level; + + /** + * Creates a new non-emitting, DEBUG level stat. + */ + public CoordinatorStat(String shortStatName) + { + this(shortStatName, null, Level.DEBUG); + } + + public CoordinatorStat(String shortName, Level level) + { + this(shortName, null, level); + } + + /** + * Creates a new emitting, DEBUG level stat. + */ + public CoordinatorStat(String shortStatName, String metricName) + { + this(shortStatName, metricName, Level.DEBUG); + } + + public CoordinatorStat(String shortStatName, String metricName, Level level) + { + this.metricName = metricName; + this.shortName = shortStatName; + this.level = level == null ? Level.DEBUG : level; + } + + /** + * Name of the metric emitted for this stat, if any. + */ + public String getMetricName() + { + return metricName; + } + + public String getShortName() + { + return shortName; + } + + public Level getLevel() + { + return level; + } + + /** + * Whether this statistic should be emitted as a metric. + */ + public boolean shouldEmit() + { + return metricName != null; + } + + @Override + public String toString() + { + return shortName; + } + + /** + * Level of coordinator stat, typically used for logging. + */ + public enum Level + { + DEBUG, INFO, ERROR + } + +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Dimension.java similarity index 58% rename from server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java rename to server/src/main/java/org/apache/druid/server/coordinator/stats/Dimension.java index 6b319c02b53..61bb6054f92 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Dimension.java @@ -17,18 +17,38 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.stats; -public class CoordinatorRuntimeParamsTestHelpers +/** + * Dimensions used while collecting or reporting coordinator run stats. + */ +public enum Dimension { - public static DruidCoordinatorRuntimeParams.Builder newBuilder() + TIER("tier"), + DATASOURCE("dataSource"), + DUTY("duty"), + DUTY_GROUP("dutyGroup"), + DESCRIPTION("description"), + SERVER("server"); + + private final String reportedName; + + Dimension(String name) { - return DruidCoordinatorRuntimeParams - .newBuilder() - .withStartTimeNanos(System.nanoTime()); + this.reportedName = name; } - private CoordinatorRuntimeParamsTestHelpers() + /** + * The name of this dimension used while emitting metrics. + */ + public String reportedName() { + return reportedName; + } + + @Override + public String toString() + { + return reportedName; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java new file mode 100644 index 00000000000..1893f86a9cd --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java @@ -0,0 +1,103 @@ +/* + * 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.druid.server.coordinator.stats; + +import java.util.Collections; +import java.util.EnumMap; +import java.util.Map; +import java.util.Objects; + +/** + * Represents a row key against which stats are reported. + */ +public class RowKey +{ + public static final RowKey EMPTY = new RowKey(Collections.emptyMap()); + + private final Map values; + private final int hashCode; + + private RowKey(Map values) + { + this.values = values; + this.hashCode = Objects.hash(values); + } + + public static Builder builder() + { + return new RowKey.Builder(); + } + + public static RowKey forTier(String tier) + { + return RowKey.builder().add(Dimension.TIER, tier).build(); + } + + public static RowKey forDatasource(String datasource) + { + return RowKey.builder().add(Dimension.DATASOURCE, datasource).build(); + } + + public Map getValues() + { + return values; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RowKey that = (RowKey) o; + return Objects.equals(values, that.values); + } + + @Override + public int hashCode() + { + return hashCode; + } + + public static class Builder + { + private final Map values = new EnumMap<>(Dimension.class); + + public Builder add(Dimension dimension, String value) + { + values.put(dimension, value); + return this; + } + + public RowKey build() + { + return new RowKey(values); + } + } + + @Override + public String toString() + { + return values == null ? "{}" : values.toString(); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java new file mode 100644 index 00000000000..791d3963dd8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java @@ -0,0 +1,141 @@ +/* + * 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.druid.server.coordinator.stats; + +/** + * List of Coordinator stats. + */ +public class Stats +{ + public static class Segments + { + // Decisions taken in a run + public static final CoordinatorStat ASSIGNED + = new CoordinatorStat("assigned", "segment/assigned/count", CoordinatorStat.Level.INFO); + public static final CoordinatorStat DROPPED + = new CoordinatorStat("dropped", "segment/dropped/count", CoordinatorStat.Level.INFO); + public static final CoordinatorStat DELETED + = new CoordinatorStat("deleted", "segment/deleted/count", CoordinatorStat.Level.INFO); + public static final CoordinatorStat MOVED + = new CoordinatorStat("moved", "segment/moved/count"); + + // Skipped decisions in a run + public static final CoordinatorStat ASSIGN_SKIPPED + = new CoordinatorStat("assignSkip", "segment/assignSkipped/count"); + public static final CoordinatorStat DROP_SKIPPED + = new CoordinatorStat("dropSkip", "segment/dropSkipped/count"); + public static final CoordinatorStat MOVE_SKIPPED + = new CoordinatorStat("moveSkip", "segment/moveSkipped/count"); + + // Current state of segments of a datasource + public static final CoordinatorStat USED + = new CoordinatorStat("usedSegments", "segment/count"); + public static final CoordinatorStat USED_BYTES + = new CoordinatorStat("usedSegmentBytes", "segment/size"); + public static final CoordinatorStat UNDER_REPLICATED + = new CoordinatorStat("underreplicated", "segment/underReplicated/count"); + public static final CoordinatorStat UNAVAILABLE + = new CoordinatorStat("unavailable", "segment/unavailable/count"); + public static final CoordinatorStat UNNEEDED + = new CoordinatorStat("unneeded", "segment/unneeded/count"); + public static final CoordinatorStat OVERSHADOWED + = new CoordinatorStat("overshadowed", "segment/overshadowed/count"); + } + + public static class SegmentQueue + { + public static final CoordinatorStat NUM_TO_LOAD + = new CoordinatorStat("numToLoad", "segment/loadQueue/count"); + public static final CoordinatorStat BYTES_TO_LOAD + = new CoordinatorStat("bytesToLoad", "segment/loadQueue/size"); + public static final CoordinatorStat NUM_TO_DROP + = new CoordinatorStat("numToDrop", "segment/dropQueue/count"); + + public static final CoordinatorStat ASSIGNED_ACTIONS + = new CoordinatorStat("assignedActions", "segment/loadQueue/assigned"); + public static final CoordinatorStat COMPLETED_ACTIONS + = new CoordinatorStat("successActions", "segment/loadQueue/success"); + public static final CoordinatorStat FAILED_ACTIONS + = new CoordinatorStat("failedActions", "segment/loadQueue/failed", CoordinatorStat.Level.ERROR); + public static final CoordinatorStat CANCELLED_ACTIONS + = new CoordinatorStat("cancelledActions", "segment/loadQueue/cancelled"); + } + + public static class Tier + { + public static final CoordinatorStat REQUIRED_CAPACITY + = new CoordinatorStat("reqdCap", "tier/required/capacity"); + public static final CoordinatorStat TOTAL_CAPACITY + = new CoordinatorStat("totalCap", "tier/total/capacity"); + public static final CoordinatorStat REPLICATION_FACTOR + = new CoordinatorStat("maxRepFactor", "tier/replication/factor"); + public static final CoordinatorStat HISTORICAL_COUNT + = new CoordinatorStat("numHistorical", "tier/historical/count"); + } + + public static class Compaction + { + public static final CoordinatorStat SUBMITTED_TASKS + = new CoordinatorStat("compactTasks", "compact/task/count"); + public static final CoordinatorStat MAX_SLOTS + = new CoordinatorStat("compactMaxSlots", "compactTask/maxSlot/count"); + public static final CoordinatorStat AVAILABLE_SLOTS + = new CoordinatorStat("compactAvlSlots", "compactTask/availableSlot/count"); + + public static final CoordinatorStat PENDING_BYTES + = new CoordinatorStat("compactPendingBytes", "segment/waitCompact/bytes"); + public static final CoordinatorStat COMPACTED_BYTES + = new CoordinatorStat("compactedBytes", "segment/compacted/bytes"); + public static final CoordinatorStat SKIPPED_BYTES + = new CoordinatorStat("compactSkipBytes", "segment/skipCompact/bytes"); + + public static final CoordinatorStat PENDING_SEGMENTS + = new CoordinatorStat("compactPendingSeg", "segment/waitCompact/count"); + public static final CoordinatorStat COMPACTED_SEGMENTS + = new CoordinatorStat("compactedSeg", "segment/compacted/count"); + public static final CoordinatorStat SKIPPED_SEGMENTS + = new CoordinatorStat("compactSkipSeg", "segment/skipCompact/count"); + + public static final CoordinatorStat PENDING_INTERVALS + = new CoordinatorStat("compactPendingIntv", "interval/waitCompact/count"); + public static final CoordinatorStat COMPACTED_INTERVALS + = new CoordinatorStat("compactedIntv", "interval/compacted/count"); + public static final CoordinatorStat SKIPPED_INTERVALS + = new CoordinatorStat("compactSkipIntv", "interval/skipCompact/count"); + } + + public static class CoordinatorRun + { + public static final CoordinatorStat DUTY_RUN_TIME + = new CoordinatorStat("dutyRunTime", "coordinator/time"); + public static final CoordinatorStat GROUP_RUN_TIME + = new CoordinatorStat("groupRunTime", "coordinator/global/time"); + } + + public static class Balancer + { + public static final CoordinatorStat RAW_COST + = new CoordinatorStat("initialCost", "segment/cost/raw"); + public static final CoordinatorStat NORMALIZATION_COST + = new CoordinatorStat("normaliznCost", "segment/cost/normalization"); + public static final CoordinatorStat NORMALIZED_COST_X_1000 + = new CoordinatorStat("normalizedCost", "segment/cost/normalized"); + } +} diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java index 488b0e70572..3fd97fcdf9d 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java @@ -121,8 +121,7 @@ public class CoordinatorDynamicConfigsResource CoordinatorDynamicConfig.CONFIG_KEY, count ) - ) - .build(); + ).build(); } catch (IllegalArgumentException e) { return Response.status(Response.Status.BAD_REQUEST) @@ -136,8 +135,7 @@ public class CoordinatorDynamicConfigsResource CoordinatorDynamicConfig.CONFIG_KEY, theInterval ) - ) - .build(); + ).build(); } } diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java index a12056336ea..7a8c3e904dc 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java @@ -19,14 +19,12 @@ package org.apache.druid.server.http; -import com.google.common.base.Function; import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.http.security.StateResourceFilter; import org.apache.druid.timeline.DataSegment; @@ -47,9 +45,7 @@ public class CoordinatorResource private final DruidCoordinator coordinator; @Inject - public CoordinatorResource( - DruidCoordinator coordinator - ) + public CoordinatorResource(DruidCoordinator coordinator) { this.coordinator = coordinator; } @@ -91,15 +87,15 @@ public class CoordinatorResource ) { if (simple != null) { - return Response.ok(coordinator.computeNumsUnavailableUsedSegmentsPerDataSource()).build(); + return Response.ok(coordinator.getDatasourceToUnavailableSegmentCount()).build(); } if (full != null) { - return computeUsingClusterView != null - ? Response.ok(coordinator.computeUnderReplicationCountsPerDataSourcePerTierUsingClusterView()).build() : - Response.ok(coordinator.computeUnderReplicationCountsPerDataSourcePerTier()).build(); + return Response.ok( + coordinator.getTierToDatasourceToUnderReplicatedCount(computeUsingClusterView != null) + ).build(); } - return Response.ok(coordinator.getLoadStatus()).build(); + return Response.ok(coordinator.getDatasourceToLoadStatus()).build(); } @GET @@ -115,28 +111,15 @@ public class CoordinatorResource return Response.ok( Maps.transformValues( coordinator.getLoadManagementPeons(), - new Function() - { - @Override - public Object apply(LoadQueuePeon input) - { - long loadSize = 0; - for (DataSegment dataSegment : input.getSegmentsToLoad()) { - loadSize += dataSegment.getSize(); - } - - long dropSize = 0; - for (DataSegment dataSegment : input.getSegmentsToDrop()) { - dropSize += dataSegment.getSize(); - } - - return new ImmutableMap.Builder<>() - .put("segmentsToLoad", input.getSegmentsToLoad().size()) - .put("segmentsToDrop", input.getSegmentsToDrop().size()) - .put("segmentsToLoadSize", loadSize) - .put("segmentsToDropSize", dropSize) - .build(); - } + input -> { + long loadSize = input.getSizeOfSegmentsToLoad(); + long dropSize = input.getSegmentsToDrop().stream().mapToLong(DataSegment::getSize).sum(); + return new ImmutableMap.Builder<>() + .put("segmentsToLoad", input.getSegmentsToLoad().size()) + .put("segmentsToDrop", input.getSegmentsToDrop().size()) + .put("segmentsToLoadSize", loadSize) + .put("segmentsToDropSize", dropSize) + .build(); } ) ).build(); @@ -149,18 +132,11 @@ public class CoordinatorResource return Response.ok( Maps.transformValues( coordinator.getLoadManagementPeons(), - new Function() - { - @Override - public Object apply(LoadQueuePeon peon) - { - return ImmutableMap - .builder() - .put("segmentsToLoad", Collections2.transform(peon.getSegmentsToLoad(), DataSegment::getId)) - .put("segmentsToDrop", Collections2.transform(peon.getSegmentsToDrop(), DataSegment::getId)) - .build(); - } - } + peon -> ImmutableMap + .builder() + .put("segmentsToLoad", Collections2.transform(peon.getSegmentsToLoad(), DataSegment::getId)) + .put("segmentsToDrop", Collections2.transform(peon.getSegmentsToDrop(), DataSegment::getId)) + .build() ) ).build(); } diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 33ff981c4bc..291bfbc8fa0 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -465,9 +465,7 @@ public class DataSourcesResource } else if (full != null) { // Calculate response for full mode Map> segmentLoadMap = - (computeUsingClusterView != null) ? - coordinator.computeUnderReplicationCountsPerDataSourcePerTierForSegmentsUsingClusterView(segments.get()) : - coordinator.computeUnderReplicationCountsPerDataSourcePerTierForSegments(segments.get()); + coordinator.getTierToDatasourceToUnderReplicatedCount(segments.get(), computeUsingClusterView != null); if (segmentLoadMap.isEmpty()) { return Response.serverError() .entity("Coordinator segment replicant lookup is not initialized yet. Try again later.") diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index 0583cc87a16..78d799b451b 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -188,23 +188,14 @@ public class MetadataResource .flatMap(t -> t.getSegments().stream()); final Set overshadowedSegments = dataSourcesSnapshot.getOvershadowedSegments(); - final Stream segmentStatus = usedSegments - .map(segment -> { - boolean isOvershadowed = overshadowedSegments.contains(segment); - Integer replicationFactor; - if (isOvershadowed) { - // If the segment is overshadowed, the replication factor won't be present in the coordinator, but we know - // that it should be 0 as we will unload it soon. - replicationFactor = 0; - } else { - replicationFactor = coordinator.getReplicationFactorForSegment(segment.getId()); - } - return new SegmentStatusInCluster( - segment, - isOvershadowed, - replicationFactor - ); - }); + final Stream segmentStatus = usedSegments.map(segment -> { + // The replication factor for unloaded segments is 0 as they will be unloaded soon + boolean isOvershadowed = overshadowedSegments.contains(segment); + Integer replicationFactor = isOvershadowed ? (Integer) 0 + : coordinator.getReplicationFactor(segment.getId()); + + return new SegmentStatusInCluster(segment, isOvershadowed, replicationFactor); + }); final Function> raGenerator = segment -> Collections .singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSegment().getDataSource())); diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java index 45dae0af0aa..a0281b27ff2 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java @@ -35,7 +35,7 @@ import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.SegmentLoadDropHandler; -import org.apache.druid.server.coordinator.HttpLoadQueuePeon; +import org.apache.druid.server.coordinator.loading.HttpLoadQueuePeon; import org.apache.druid.server.http.security.StateResourceFilter; import javax.annotation.Nullable; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java index 1512704ec0f..91ee44d1ce1 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java @@ -40,9 +40,7 @@ public class TestUsedSegmentChecker implements UsedSegmentChecker @Override public Set findUsedSegments(Set identifiers) { - final SegmentTimeline timeline = new SegmentTimeline(); - timeline.addSegments(pushedSegments.iterator()); - + final SegmentTimeline timeline = SegmentTimeline.forSegments(pushedSegments); final Set retVal = new HashSet<>(); for (SegmentIdWithShardSpec identifier : identifiers) { for (TimelineObjectHolder holder : timeline.lookup(identifier.getInterval())) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java index 06c43e9a68e..b117b96520f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java @@ -28,7 +28,11 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; +import org.apache.druid.server.coordinator.duty.BalanceSegments; import org.apache.druid.server.coordinator.duty.RunRules; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.rules.PeriodLoadRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.timeline.DataSegment; @@ -42,7 +46,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Map; /** * TODO convert benchmarks to JMH @@ -50,7 +53,7 @@ import java.util.Map; public class BalanceSegmentsProfiler { private static final int MAX_SEGMENTS_TO_MOVE = 5; - private DruidCoordinator coordinator; + private SegmentLoadQueueManager loadQueueManager; private ImmutableDruidServer druidServer1; private ImmutableDruidServer druidServer2; List segments = new ArrayList<>(); @@ -62,7 +65,7 @@ public class BalanceSegmentsProfiler @Before public void setUp() { - coordinator = EasyMock.createMock(DruidCoordinator.class); + loadQueueManager = new SegmentLoadQueueManager(null, null, null); druidServer1 = EasyMock.createMock(ImmutableDruidServer.class); druidServer2 = EasyMock.createMock(ImmutableDruidServer.class); emitter = EasyMock.createMock(ServiceEmitter.class); @@ -80,17 +83,6 @@ public class BalanceSegmentsProfiler EasyMock.expect(manager.getRulesWithDefault(EasyMock.anyObject())).andReturn(rules).anyTimes(); EasyMock.replay(manager); - coordinator.moveSegment( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - ); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(coordinator); - - Map peonMap = new HashMap<>(); List serverHolderList = new ArrayList<>(); List segments = new ArrayList<>(); for (int i = 0; i < numSegments; i++) { @@ -126,19 +118,16 @@ public class BalanceSegmentsProfiler EasyMock.replay(server); LoadQueuePeon peon = new LoadQueuePeonTester(); - peonMap.put(Integer.toString(i), peon); serverHolderList.add(new ServerHolder(server, peon)); } - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() + DruidCluster druidCluster = DruidCluster + .builder() .addTier("normal", serverHolderList.toArray(new ServerHolder[0])) .build(); - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() + DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc()) .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false)) - .withLoadManagementPeons(peonMap) .withUsedSegmentsInTest(segments) .withDynamicConfigs( CoordinatorDynamicConfig @@ -148,13 +137,13 @@ public class BalanceSegmentsProfiler .withReplicationThrottleLimit(5) .build() ) + .withSegmentAssignerUsing(loadQueueManager) .withEmitter(emitter) .withDatabaseRuleManager(manager) - .withReplicationManager(new ReplicationThrottler(2, 500, false)) .build(); - BalanceSegmentsTester tester = new BalanceSegmentsTester(coordinator); - RunRules runner = new RunRules(coordinator); + BalanceSegments tester = new BalanceSegments(); + RunRules runner = new RunRules(); watch.start(); DruidCoordinatorRuntimeParams balanceParams = tester.run(params); DruidCoordinatorRuntimeParams assignParams = runner.run(params); @@ -183,21 +172,11 @@ public class BalanceSegmentsProfiler EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); - coordinator.moveSegment( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - ); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(coordinator); - - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() + DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc()) .withDruidCluster( - DruidClusterBuilder - .newBuilder() + DruidCluster + .builder() .addTier( "normal", new ServerHolder(druidServer1, fromPeon), @@ -205,11 +184,11 @@ public class BalanceSegmentsProfiler ) .build() ) - .withLoadManagementPeons(ImmutableMap.of("from", fromPeon, "to", toPeon)) .withUsedSegmentsInTest(segments) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()) + .withSegmentAssignerUsing(loadQueueManager) .build(); - BalanceSegmentsTester tester = new BalanceSegmentsTester(coordinator); + BalanceSegments tester = new BalanceSegments(); watch.start(); DruidCoordinatorRuntimeParams balanceParams = tester.run(params); System.out.println(watch.stop()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java deleted file mode 100644 index 740eb76d263..00000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java +++ /dev/null @@ -1,814 +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.druid.server.coordinator; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.client.ImmutableDruidServerTests; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NoneShardSpec; -import org.easymock.EasyMock; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -public class BalanceSegmentsTest -{ - private static final int MAX_SEGMENTS_TO_MOVE = 5; - private DruidCoordinator coordinator; - private ImmutableDruidServer druidServer1; - private ImmutableDruidServer druidServer2; - private ImmutableDruidServer druidServer3; - private ImmutableDruidServer druidServer4; - private List druidServers; - private LoadQueuePeonTester peon1; - private LoadQueuePeonTester peon2; - private LoadQueuePeonTester peon3; - private LoadQueuePeonTester peon4; - private List peons; - private DataSegment segment1; - private DataSegment segment2; - private DataSegment segment3; - private DataSegment segment4; - private DataSegment segment5; - private List segments; - private ListeningExecutorService balancerStrategyExecutor; - private BalancerStrategy balancerStrategy; - private Set broadcastDatasources; - - @Before - public void setUp() - { - coordinator = EasyMock.createMock(DruidCoordinator.class); - druidServer1 = EasyMock.createMock(ImmutableDruidServer.class); - druidServer2 = EasyMock.createMock(ImmutableDruidServer.class); - druidServer3 = EasyMock.createMock(ImmutableDruidServer.class); - druidServer4 = EasyMock.createMock(ImmutableDruidServer.class); - segment1 = EasyMock.createMock(DataSegment.class); - segment2 = EasyMock.createMock(DataSegment.class); - segment3 = EasyMock.createMock(DataSegment.class); - segment4 = EasyMock.createMock(DataSegment.class); - segment5 = EasyMock.createMock(DataSegment.class); - - DateTime start1 = DateTimes.of("2012-01-01"); - DateTime start2 = DateTimes.of("2012-02-01"); - DateTime version = DateTimes.of("2012-03-01"); - segment1 = new DataSegment( - "datasource1", - new Interval(start1, start1.plusHours(1)), - version.toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 0, - 11L - ); - segment2 = new DataSegment( - "datasource1", - new Interval(start2, start2.plusHours(1)), - version.toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 0, - 7L - ); - segment3 = new DataSegment( - "datasource2", - new Interval(start1, start1.plusHours(1)), - version.toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 0, - 4L - ); - segment4 = new DataSegment( - "datasource2", - new Interval(start2, start2.plusHours(1)), - version.toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 0, - 8L - ); - segment5 = new DataSegment( - "datasourceBroadcast", - new Interval(start2, start2.plusHours(1)), - version.toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 0, - 8L - ); - - segments = new ArrayList<>(); - segments.add(segment1); - segments.add(segment2); - segments.add(segment3); - segments.add(segment4); - segments.add(segment5); - - peon1 = new LoadQueuePeonTester(); - peon2 = new LoadQueuePeonTester(); - peon3 = new LoadQueuePeonTester(); - peon4 = new LoadQueuePeonTester(); - - druidServers = ImmutableList.of(druidServer1, druidServer2, druidServer3, druidServer4); - peons = ImmutableList.of(peon1, peon2, peon3, peon4); - - balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d")); - balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor); - - broadcastDatasources = Collections.singleton("datasourceBroadcast"); - } - - @After - public void tearDown() - { - EasyMock.verify(coordinator); - EasyMock.verify(druidServer1); - EasyMock.verify(druidServer2); - EasyMock.verify(druidServer3); - EasyMock.verify(druidServer4); - balancerStrategyExecutor.shutdownNow(); - } - - @Test - public void testMoveToEmptyServerBalancer() - { - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); - mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer3); - EasyMock.replay(druidServer4); - - // Mock stuff that the coordinator needs - mockCoordinator(coordinator); - - BalancerStrategy predefinedPickOrderStrategy = new PredefinedPickOrderBalancerStrategy( - balancerStrategy, - ImmutableList.of( - new BalancerSegmentHolder(druidServer1, segment1), - new BalancerSegmentHolder(druidServer1, segment2), - new BalancerSegmentHolder(druidServer1, segment3), - new BalancerSegmentHolder(druidServer1, segment4) - ) - ); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2), - ImmutableList.of(peon1, peon2) - ) - .withBalancerStrategy(predefinedPickOrderStrategy) - .withBroadcastDatasources(broadcastDatasources) - .build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - Assert.assertEquals(3, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - } - - /** - * Server 1 has 2 segments. - * Server 2 (decommissioning) has 2 segments. - * Server 3 is empty. - * Decommissioning percent is 60. - * Max segments to move is 3. - * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1. - */ - @Test - public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove() - { - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, Arrays.asList(segment1, segment2)); - mockDruidServer(druidServer2, "2", "normal", 30L, 100L, Arrays.asList(segment3, segment4)); - mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer4); - - mockCoordinator(coordinator); - - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect( - strategy.pickSegmentsToMove( - ImmutableList.of( - new ServerHolder(druidServer2, peon2, true) - ), - broadcastDatasources, - 2 - ) - ).andReturn( - ImmutableList.of( - new BalancerSegmentHolder(druidServer2, segment3), - new BalancerSegmentHolder(druidServer2, segment4) - ).iterator() - ); - - EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt())) - .andReturn( - ImmutableList.of( - new BalancerSegmentHolder(druidServer1, segment1), - new BalancerSegmentHolder(druidServer1, segment2)).iterator()); - - EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())) - .andReturn(new ServerHolder(druidServer3, peon3)) - .anyTimes(); - EasyMock.replay(strategy); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2, druidServer3), - ImmutableList.of(peon1, peon2, peon3), - ImmutableList.of(false, true, false) - ) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder() - .withMaxSegmentsToMove(3) - .withDecommissioningMaxPercentOfMaxSegmentsToMove(60) - .build() // ceil(3 * 0.6) = 2 segments from decommissioning servers - ) - .withBalancerStrategy(strategy) - .withBroadcastDatasources(broadcastDatasources) - .build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - EasyMock.verify(strategy); - Assert.assertEquals(3L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - Assert.assertEquals( - ImmutableSet.of(segment1, segment3, segment4), - peon3.getSegmentsToLoad() - ); - } - - @Test - public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove() - { - DruidCoordinatorRuntimeParams params = setupParamsForDecommissioningMaxPercentOfMaxSegmentsToMove(0); - params = new BalanceSegmentsTester(coordinator).run(params); - Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - Assert.assertEquals(ImmutableSet.of(segment1), peon3.getSegmentsToLoad()); - } - - @Test - public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove() - { - DruidCoordinatorRuntimeParams params = setupParamsForDecommissioningMaxPercentOfMaxSegmentsToMove(10); - params = new BalanceSegmentsTester(coordinator).run(params); - Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - Assert.assertEquals(ImmutableSet.of(segment1), peon3.getSegmentsToLoad()); - } - - /** - * Should balance segments as usual (ignoring percent) with empty decommissioningNodes. - */ - @Test - public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning() - { - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, Arrays.asList(segment1, segment2)); - mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Arrays.asList(segment3, segment4)); - mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer4); - - mockCoordinator(coordinator); - - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect( - strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt())) - .andReturn( - ImmutableList.of( - new BalancerSegmentHolder(druidServer1, segment2), - new BalancerSegmentHolder(druidServer2, segment3), - new BalancerSegmentHolder(druidServer2, segment4)).iterator()); - - EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())) - .andReturn(new ServerHolder(druidServer3, peon3)) - .anyTimes(); - EasyMock.replay(strategy); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2, druidServer3), - ImmutableList.of(peon1, peon2, peon3), - ImmutableList.of(false, false, false) - ) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder() - .withMaxSegmentsToMove(3) - .withDecommissioningMaxPercentOfMaxSegmentsToMove(9) - .build() - ) - .withBalancerStrategy(strategy) - .withBroadcastDatasources(broadcastDatasources) - .build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - EasyMock.verify(strategy); - Assert.assertEquals(3L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - Assert.assertEquals( - ImmutableSet.of(segment2, segment3, segment4), - peon3.getSegmentsToLoad() - ); - } - - /** - * Shouldn't move segments to a decommissioning server. - */ - @Test - public void testMoveToDecommissioningServer() - { - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); - mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer3); - EasyMock.replay(druidServer4); - - mockCoordinator(coordinator); - - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt())) - .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer1, segment1)).iterator()) - .anyTimes(); - EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())).andAnswer(() -> { - List holders = (List) EasyMock.getCurrentArguments()[1]; - return holders.get(0); - }).anyTimes(); - EasyMock.replay(strategy); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2), - ImmutableList.of(peon1, peon2), - ImmutableList.of(false, true) - ) - .withBalancerStrategy(strategy) - .withBroadcastDatasources(broadcastDatasources) - .build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - EasyMock.verify(strategy); - Assert.assertEquals(0, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - } - - @Test - public void testMoveFromDecommissioningServer() - { - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); - mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer3); - EasyMock.replay(druidServer4); - - mockCoordinator(coordinator); - - ServerHolder holder2 = new ServerHolder(druidServer2, peon2, false); - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt())) - .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer1, segment1)).iterator()) - .once(); - EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())) - .andReturn(holder2) - .once(); - EasyMock.replay(strategy); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2), - ImmutableList.of(peon1, peon2), - ImmutableList.of(true, false) - ) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build()) - .withBalancerStrategy(strategy) - .withBroadcastDatasources(broadcastDatasources) - .build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - EasyMock.verify(strategy); - Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - Assert.assertEquals(0, peon1.getNumberOfSegmentsInQueue()); - Assert.assertEquals(1, peon2.getNumberOfSegmentsInQueue()); - } - - @Test - public void testMoveMaxLoadQueueServerBalancer() - { - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); - mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer3); - EasyMock.replay(druidServer4); - - // Mock stuff that the coordinator needs - mockCoordinator(coordinator); - - BalancerStrategy predefinedPickOrderStrategy = new PredefinedPickOrderBalancerStrategy( - balancerStrategy, - ImmutableList.of( - new BalancerSegmentHolder(druidServer1, segment1), - new BalancerSegmentHolder(druidServer1, segment2), - new BalancerSegmentHolder(druidServer1, segment3), - new BalancerSegmentHolder(druidServer1, segment4) - ) - ); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2), - ImmutableList.of(peon1, peon2) - ) - .withBalancerStrategy(predefinedPickOrderStrategy) - .withBroadcastDatasources(broadcastDatasources) - .withDynamicConfigs( - CoordinatorDynamicConfig - .builder() - .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) - .withMaxSegmentsInNodeLoadingQueue(1) - .build() - ) - .build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - - // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1 - Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - } - - @Test - public void testMoveSameSegmentTwice() - { - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); - mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer3); - EasyMock.replay(druidServer4); - - // Mock stuff that the coordinator needs - mockCoordinator(coordinator); - - BalancerStrategy predefinedPickOrderStrategy = new PredefinedPickOrderBalancerStrategy( - balancerStrategy, - ImmutableList.of( - new BalancerSegmentHolder(druidServer1, segment1) - ) - ); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2), - ImmutableList.of(peon1, peon2) - ) - .withBalancerStrategy(predefinedPickOrderStrategy) - .withBroadcastDatasources(broadcastDatasources) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( - 2 - ).build() - ) - .build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - } - - @Test - public void testRun1() - { - // Mock some servers of different usages - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); - mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer3); - EasyMock.replay(druidServer4); - - // Mock stuff that the coordinator needs - mockCoordinator(coordinator); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2), - ImmutableList.of(peon1, peon2) - ).build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0); - } - - @Test - public void testRun2() - { - // Mock some servers of different usages - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); - mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); - mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList()); - mockDruidServer(druidServer4, "4", "normal", 0L, 100L, Collections.emptyList()); - - // Mock stuff that the coordinator needs - mockCoordinator(coordinator); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(druidServers, peons).build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0); - } - - /** - * Testing that the dynamic coordinator config value, percentOfSegmentsToConsiderPerMove, is honored when calling - * out to pickSegmentToMove. This config limits the number of segments that are considered when looking for a segment - * to move. - */ - @Test - public void testThatDynamicConfigIsHonoredWhenPickingSegmentToMove() - { - mockDruidServer(druidServer1, "1", "normal", 50L, 100L, Arrays.asList(segment1, segment2)); - mockDruidServer(druidServer2, "2", "normal", 30L, 100L, Arrays.asList(segment3, segment4)); - mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer4); - - mockCoordinator(coordinator); - - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - - // Move from non-decomissioning servers - EasyMock.expect( - strategy.pickSegmentsToMove( - ImmutableList.of( - new ServerHolder(druidServer3, peon3, false), - new ServerHolder(druidServer2, peon2, false), - new ServerHolder(druidServer1, peon1, false) - ), - broadcastDatasources, - 40.0 - ) - ) - .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer2, segment3)).iterator()); - - EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())) - .andReturn(new ServerHolder(druidServer3, peon3)) - .anyTimes(); - EasyMock.replay(strategy); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2, druidServer3), - ImmutableList.of(peon1, peon2, peon3), - ImmutableList.of(false, false, false) - ) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder() - .withMaxSegmentsToMove(1) - .withUseBatchedSegmentSampler(false) - .withPercentOfSegmentsToConsiderPerMove(40) - .build() - ) - .withBalancerStrategy(strategy) - .withBroadcastDatasources(broadcastDatasources) - .build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - EasyMock.verify(strategy); - Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - Assert.assertEquals(ImmutableSet.of(segment3), peon3.getSegmentsToLoad()); - } - - @Test - public void testUseBatchedSegmentSampler() - { - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); - mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); - mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList()); - mockDruidServer(druidServer4, "4", "normal", 0L, 100L, Collections.emptyList()); - - mockCoordinator(coordinator); - - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(druidServers, peons) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder() - .withMaxSegmentsToMove(2) - .withUseBatchedSegmentSampler(true) - .build() - ) - .withBroadcastDatasources(broadcastDatasources) - .build(); - - params = new BalanceSegmentsTester(coordinator).run(params); - Assert.assertEquals(2L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); - } - - private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( - List druidServers, - List peons - ) - { - return defaultRuntimeParamsBuilder( - druidServers, - peons, - druidServers.stream().map(s -> false).collect(Collectors.toList()) - ); - } - - private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( - List druidServers, - List peons, - List decommissioning - ) - { - return CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster( - DruidClusterBuilder - .newBuilder() - .addTier( - "normal", - IntStream - .range(0, druidServers.size()) - .mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i), decommissioning.get(i))) - .toArray(ServerHolder[]::new) - ) - .build() - ) - .withLoadManagementPeons( - IntStream - .range(0, peons.size()) - .boxed() - .collect(Collectors.toMap(i -> String.valueOf(i + 1), peons::get)) - ) - .withUsedSegmentsInTest(segments) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()) - .withBroadcastDatasources(broadcastDatasources) - .withBalancerStrategy(balancerStrategy); - } - - private static void mockDruidServer( - ImmutableDruidServer druidServer, - String name, - String tier, - long currentSize, - long maxSize, - List segments - ) - { - EasyMock.expect(druidServer.getName()).andReturn(name).anyTimes(); - EasyMock.expect(druidServer.getTier()).andReturn(tier).anyTimes(); - EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce(); - EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).atLeastOnce(); - ImmutableDruidServerTests.expectSegments(druidServer, segments); - EasyMock.expect(druidServer.getHost()).andReturn(name).anyTimes(); - EasyMock.expect(druidServer.getType()).andReturn(ServerType.HISTORICAL).anyTimes(); - if (!segments.isEmpty()) { - segments.forEach( - s -> EasyMock.expect(druidServer.getSegment(s.getId())).andReturn(s).anyTimes() - ); - } - EasyMock.expect(druidServer.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); - EasyMock.replay(druidServer); - } - - private static void mockCoordinator(DruidCoordinator coordinator) - { - coordinator.moveSegment( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - ); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(coordinator); - } - - private static class PredefinedPickOrderBalancerStrategy implements BalancerStrategy - { - private final BalancerStrategy delegate; - private final List pickOrder; - private final AtomicInteger pickCounter = new AtomicInteger(0); - - PredefinedPickOrderBalancerStrategy( - BalancerStrategy delegate, - List pickOrder - ) - { - this.delegate = delegate; - this.pickOrder = pickOrder; - } - - @Override - public ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List serverHolders) - { - return delegate.findNewSegmentHomeBalancer(proposalSegment, serverHolders); - } - - @Override - public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List serverHolders) - { - return delegate.findNewSegmentHomeReplicator(proposalSegment, serverHolders); - } - - @Override - public Iterator pickSegmentsToMove( - List serverHolders, - Set broadcastDatasources, - int numberOfSegments - ) - { - return pickOrder.iterator(); - } - - @Override - public Iterator pickSegmentsToMove( - List serverHolders, - Set broadcastDatasources, - double percentOfSegmentsToConsider - ) - { - return pickOrder.iterator(); - } - - @Override - public void emitStats(String tier, CoordinatorStats stats, List serverHolderList) - { - delegate.emitStats(tier, stats, serverHolderList); - } - } - - private DruidCoordinatorRuntimeParams setupParamsForDecommissioningMaxPercentOfMaxSegmentsToMove(int percent) - { - mockDruidServer(druidServer1, "1", "normal", 30L, 100L, Arrays.asList(segment1, segment3)); - mockDruidServer(druidServer2, "2", "normal", 30L, 100L, Arrays.asList(segment2, segment3)); - mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList()); - - EasyMock.replay(druidServer4); - - mockCoordinator(coordinator); - - // either decommissioning servers list or acitve ones (ie servers list is [2] or [1, 3]) - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect( - strategy.pickSegmentsToMove( - ImmutableList.of( - new ServerHolder(druidServer2, peon2, true) - ), - broadcastDatasources, - 100.0 - ) - ).andReturn( - ImmutableList.of(new BalancerSegmentHolder(druidServer2, segment2)).iterator() - ); - EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt())) - .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer1, segment1)).iterator()); - EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())) - .andReturn(new ServerHolder(druidServer3, peon3)) - .anyTimes(); - EasyMock.replay(strategy); - - return defaultRuntimeParamsBuilder( - ImmutableList.of(druidServer1, druidServer2, druidServer3), - ImmutableList.of(peon1, peon2, peon3), - ImmutableList.of(false, true, false) - ) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder() - .withMaxSegmentsToMove(1) - .withDecommissioningMaxPercentOfMaxSegmentsToMove(percent) - .build() - ) - .withBalancerStrategy(strategy) - .withBroadcastDatasources(broadcastDatasources) - .build(); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTester.java b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTester.java deleted file mode 100644 index 011b01cf67e..00000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTester.java +++ /dev/null @@ -1,76 +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.druid.server.coordinator; - -import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.coordinator.duty.BalanceSegments; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; - -public class BalanceSegmentsTester extends BalanceSegments -{ - - public BalanceSegmentsTester(DruidCoordinator coordinator) - { - super(coordinator); - } - - @Override - protected boolean moveSegment( - final BalancerSegmentHolder segment, - final ImmutableDruidServer toServer, - final DruidCoordinatorRuntimeParams params - ) - { - final String toServerName = toServer.getName(); - final LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServerName); - - final String fromServerName = segment.getFromServer().getName(); - final DataSegment segmentToMove = segment.getSegment(); - final SegmentId segmentId = segmentToMove.getId(); - - if (!toPeon.getSegmentsToLoad().contains(segmentToMove) && - (toServer.getSegment(segmentId) == null) && - new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) { - log.info( - "Moving [%s] from [%s] to [%s]", - segmentId, - fromServerName, - toServerName - ); - try { - final LoadQueuePeon loadPeon = params.getLoadManagementPeons().get(toServerName); - - loadPeon.loadSegment(segment.getSegment(), success -> {}); - - final LoadQueuePeon dropPeon = params.getLoadManagementPeons().get(fromServerName); - dropPeon.markSegmentToDrop(segment.getSegment()); - - currentlyMovingSegments.get("normal").put(segmentId, segment); - return true; - } - catch (Exception e) { - log.info(e, StringUtils.format("[%s] : Moving exception", segmentId)); - } - } - return false; - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java new file mode 100644 index 00000000000..33540800693 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java @@ -0,0 +1,211 @@ +/* + * 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.druid.server.coordinator; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.CoordinatorStat; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class CoordinatorRunStatsTest +{ + private static final CoordinatorStat STAT_1 = new CoordinatorStat("stat1", "s1"); + private static final CoordinatorStat STAT_2 = new CoordinatorStat("stat2", "s2"); + private static final CoordinatorStat STAT_3 = new CoordinatorStat("stat3", "s3"); + + private static final CoordinatorStat DEBUG_STAT_1 + = new CoordinatorStat("debugStat1", CoordinatorStat.Level.DEBUG); + private static final CoordinatorStat DEBUG_STAT_2 + = new CoordinatorStat("debugStat2", CoordinatorStat.Level.DEBUG); + + private CoordinatorRunStats stats; + + @Before + public void setUp() + { + stats = new CoordinatorRunStats(); + } + + @After + public void tearDown() + { + stats = null; + } + + @Test + public void testAdd() + { + Assert.assertEquals(0, stats.get(STAT_1)); + stats.add(STAT_1, 1); + Assert.assertEquals(1, stats.get(STAT_1)); + stats.add(STAT_1, -11); + Assert.assertEquals(-10, stats.get(STAT_1)); + } + + @Test + public void testAddForRowKey() + { + stats.add(STAT_1, Key.TIER_1, 1); + stats.add(STAT_1, Key.TIER_2, 1); + stats.add(STAT_1, Key.TIER_1, -5); + stats.add(STAT_2, Key.TIER_1, 1); + stats.add(STAT_1, Key.TIER_2, 1); + + Assert.assertFalse(stats.hasStat(STAT_3)); + + Assert.assertEquals(-4, stats.get(STAT_1, Key.TIER_1)); + Assert.assertEquals(2, stats.get(STAT_1, Key.TIER_2)); + Assert.assertEquals(1, stats.get(STAT_2, Key.TIER_1)); + } + + @Test + public void testGetSnapshotAndReset() + { + stats.add(STAT_1, 1); + stats.add(STAT_2, 3); + stats.add(STAT_1, Key.TIER_1, 5); + stats.add(STAT_1, Key.DUTY_1, 7); + + final CoordinatorRunStats firstFlush = stats.getSnapshotAndReset(); + Assert.assertEquals(1, firstFlush.get(STAT_1)); + Assert.assertEquals(3, firstFlush.get(STAT_2)); + Assert.assertEquals(5, firstFlush.get(STAT_1, Key.TIER_1)); + Assert.assertEquals(7, firstFlush.get(STAT_1, Key.DUTY_1)); + + Assert.assertEquals(0, stats.rowCount()); + + stats.add(STAT_1, 7); + stats.add(STAT_1, Key.TIER_1, 5); + stats.add(STAT_2, Key.DUTY_1, 3); + stats.add(STAT_3, Key.TIER_1, 1); + + final CoordinatorRunStats secondFlush = stats.getSnapshotAndReset(); + + Assert.assertEquals(7, secondFlush.get(STAT_1)); + Assert.assertEquals(5, secondFlush.get(STAT_1, Key.TIER_1)); + Assert.assertEquals(3, secondFlush.get(STAT_2, Key.DUTY_1)); + Assert.assertEquals(1, secondFlush.get(STAT_3, Key.TIER_1)); + + Assert.assertEquals(0, stats.rowCount()); + } + + @Test + public void testUpdateMax() + { + stats.updateMax(STAT_1, Key.TIER_1, 2); + stats.updateMax(STAT_1, Key.TIER_1, 6); + stats.updateMax(STAT_1, Key.TIER_1, 5); + + stats.updateMax(STAT_2, Key.TIER_1, 5); + stats.updateMax(STAT_2, Key.TIER_1, 4); + stats.updateMax(STAT_2, Key.TIER_1, 5); + + stats.updateMax(STAT_1, Key.TIER_2, 7); + stats.updateMax(STAT_1, Key.TIER_2, 9); + stats.updateMax(STAT_1, Key.TIER_2, 10); + + Assert.assertFalse(stats.hasStat(STAT_3)); + + Assert.assertEquals(6, stats.get(STAT_1, Key.TIER_1)); + Assert.assertEquals(5, stats.get(STAT_2, Key.TIER_1)); + Assert.assertEquals(10, stats.get(STAT_1, Key.TIER_2)); + } + + @Test + public void testAddToDutyStat() + { + stats.add(STAT_1, Key.DUTY_1, 1); + stats.add(STAT_1, Key.DUTY_2, 1); + stats.add(STAT_1, Key.DUTY_1, -5); + stats.add(STAT_2, Key.DUTY_1, 1); + stats.add(STAT_1, Key.DUTY_2, 1); + + Assert.assertFalse(stats.hasStat(STAT_3)); + Assert.assertEquals(-4, stats.get(STAT_1, Key.DUTY_1)); + Assert.assertEquals(2, stats.get(STAT_1, Key.DUTY_2)); + Assert.assertEquals(1, stats.get(STAT_2, Key.DUTY_1)); + } + + @Test + public void testForEachStat() + { + final Map expected = ImmutableMap.of( + "duty1", 1L, + "duty2", 2L, + "duty3", 3L + ); + expected.forEach( + (duty, count) -> + stats.add(STAT_1, RowKey.builder().add(Dimension.DUTY, duty).build(), count) + ); + + final Map actual = new HashMap<>(); + stats.forEachStat( + (dimensionValues, stat, value) -> { + if (stat.equals(STAT_1)) { + actual.put(dimensionValues.get(Dimension.DUTY), value); + } + } + ); + Assert.assertEquals(expected, actual); + } + + @Test + public void testAddWithDebugDimensions() + { + stats.add(DEBUG_STAT_1, 1); + Assert.assertFalse(stats.hasStat(DEBUG_STAT_1)); + + stats.add(DEBUG_STAT_1, Key.TIER_1, 1); + Assert.assertFalse(stats.hasStat(DEBUG_STAT_1)); + + final CoordinatorRunStats debugStats + = new CoordinatorRunStats(Key.TIER_1.getValues()); + debugStats.add(DEBUG_STAT_1, 1); + Assert.assertFalse(stats.hasStat(DEBUG_STAT_1)); + + debugStats.add(DEBUG_STAT_1, Key.TIER_1, 1); + Assert.assertTrue(debugStats.hasStat(DEBUG_STAT_1)); + + debugStats.addToDatasourceStat(DEBUG_STAT_2, "wiki", 1); + Assert.assertFalse(debugStats.hasStat(DEBUG_STAT_2)); + } + + /** + * Dimension keys for reporting stats. + */ + private static class Key + { + static final RowKey TIER_1 = RowKey.forTier("tier1"); + static final RowKey TIER_2 = RowKey.forTier("tier2"); + + static final RowKey DUTY_1 = RowKey.builder().add(Dimension.DUTY, "duty1").build(); + static final RowKey DUTY_2 = RowKey.builder().add(Dimension.DUTY, "duty2").build(); + } + +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorStatsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorStatsTest.java deleted file mode 100644 index 00dc0949cc0..00000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorStatsTest.java +++ /dev/null @@ -1,233 +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.druid.server.coordinator; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Sets; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; - -public class CoordinatorStatsTest -{ - private CoordinatorStats stats; - - @Before - public void setUp() - { - stats = new CoordinatorStats(); - } - - @After - public void tearDown() - { - stats = null; - } - - @Test - public void addToGlobalStat() - { - Assert.assertEquals(0, stats.getGlobalStat("stats")); - stats.addToGlobalStat("stats", 1); - Assert.assertEquals(1, stats.getGlobalStat("stats")); - stats.addToGlobalStat("stats", -11); - Assert.assertEquals(-10, stats.getGlobalStat("stats")); - } - - @Test(expected = NullPointerException.class) - public void testAddToTieredStatNonexistentStat() - { - stats.getTieredStat("stat", "tier"); - } - - @Test - public void testAddToTieredStat() - { - Assert.assertFalse(stats.hasPerTierStats()); - stats.addToTieredStat("stat1", "tier1", 1); - stats.addToTieredStat("stat1", "tier2", 1); - stats.addToTieredStat("stat1", "tier1", -5); - stats.addToTieredStat("stat2", "tier1", 1); - stats.addToTieredStat("stat1", "tier2", 1); - Assert.assertTrue(stats.hasPerTierStats()); - - Assert.assertEquals( - Sets.newHashSet("tier1", "tier2"), - stats.getTiers("stat1") - ); - Assert.assertEquals( - Sets.newHashSet("tier1"), - stats.getTiers("stat2") - ); - Assert.assertTrue(stats.getTiers("stat3").isEmpty()); - - Assert.assertEquals(-4, stats.getTieredStat("stat1", "tier1")); - Assert.assertEquals(2, stats.getTieredStat("stat1", "tier2")); - Assert.assertEquals(1, stats.getTieredStat("stat2", "tier1")); - } - - @Test - public void testForEachTieredStat() - { - final Map expected = ImmutableMap.of( - "tier1", 1L, - "tier2", 2L, - "tier3", 3L - ); - final Map actual = new HashMap<>(); - - expected.forEach( - (tier, count) -> stats.addToTieredStat("stat", tier, count) - ); - - stats.forEachTieredStat("stat0", (tier, count) -> Assert.fail()); - stats.forEachTieredStat("stat", actual::put); - - Assert.assertEquals(expected, actual); - } - - - @Test - public void testAccumulate() - { - stats.addToGlobalStat("stat1", 1); - stats.addToGlobalStat("stat2", 1); - stats.addToTieredStat("stat1", "tier1", 1); - stats.addToTieredStat("stat1", "tier2", 1); - stats.addToTieredStat("stat2", "tier1", 1); - stats.addToDutyStat("stat1", "duty1", 1); - stats.addToDutyStat("stat1", "duty2", 1); - stats.addToDutyStat("stat2", "duty1", 1); - - final CoordinatorStats stats2 = new CoordinatorStats(); - stats2.addToGlobalStat("stat1", 1); - stats2.addToTieredStat("stat1", "tier2", 1); - stats2.addToTieredStat("stat2", "tier2", 1); - stats2.addToTieredStat("stat3", "tier1", 1); - stats2.addToDutyStat("stat1", "duty2", 1); - stats2.addToDutyStat("stat2", "duty2", 1); - stats2.addToDutyStat("stat3", "duty1", 1); - - stats.accumulate(stats2); - - Assert.assertEquals(2, stats.getGlobalStat("stat1")); - Assert.assertEquals(1, stats.getGlobalStat("stat2")); - Assert.assertEquals(1, stats.getTieredStat("stat1", "tier1")); - Assert.assertEquals(2, stats.getTieredStat("stat1", "tier2")); - Assert.assertEquals(1, stats.getTieredStat("stat2", "tier1")); - Assert.assertEquals(1, stats.getTieredStat("stat2", "tier2")); - Assert.assertEquals(1, stats.getTieredStat("stat3", "tier1")); - Assert.assertEquals(1, stats.getDutyStat("stat1", "duty1")); - Assert.assertEquals(2, stats.getDutyStat("stat1", "duty2")); - Assert.assertEquals(1, stats.getDutyStat("stat2", "duty1")); - Assert.assertEquals(1, stats.getDutyStat("stat2", "duty2")); - Assert.assertEquals(1, stats.getDutyStat("stat3", "duty1")); - } - - @Test - public void testAccumulateMaxToTieredStat() - { - Assert.assertFalse(stats.hasPerTierStats()); - stats.accumulateMaxTieredStat("stat1", "tier1", 2); - stats.accumulateMaxTieredStat("stat1", "tier1", 6); - stats.accumulateMaxTieredStat("stat1", "tier1", 5); - - stats.accumulateMaxTieredStat("stat2", "tier1", 5); - stats.accumulateMaxTieredStat("stat2", "tier1", 4); - stats.accumulateMaxTieredStat("stat2", "tier1", 5); - - stats.accumulateMaxTieredStat("stat1", "tier2", 7); - stats.accumulateMaxTieredStat("stat1", "tier2", 9); - stats.accumulateMaxTieredStat("stat1", "tier2", 10); - - Assert.assertTrue(stats.hasPerTierStats()); - - Assert.assertEquals( - Sets.newHashSet("tier1", "tier2"), - stats.getTiers("stat1") - ); - Assert.assertEquals( - Sets.newHashSet("tier1"), - stats.getTiers("stat2") - ); - Assert.assertTrue(stats.getTiers("stat3").isEmpty()); - - Assert.assertEquals(6, stats.getTieredStat("stat1", "tier1")); - Assert.assertEquals(5, stats.getTieredStat("stat2", "tier1")); - Assert.assertEquals(10, stats.getTieredStat("stat1", "tier2")); - - } - - @Test(expected = NullPointerException.class) - public void testGetNonexistentDutyStat() - { - stats.getDutyStat("stat", "duty"); - } - - @Test - public void testAddToDutyStat() - { - Assert.assertFalse(stats.hasPerDutyStats()); - stats.addToDutyStat("stat1", "duty1", 1); - stats.addToDutyStat("stat1", "duty2", 1); - stats.addToDutyStat("stat1", "duty1", -5); - stats.addToDutyStat("stat2", "duty1", 1); - stats.addToDutyStat("stat1", "duty2", 1); - Assert.assertTrue(stats.hasPerDutyStats()); - - Assert.assertEquals( - Sets.newHashSet("duty1", "duty2"), - stats.getDuties("stat1") - ); - Assert.assertEquals( - Sets.newHashSet("duty1"), - stats.getDuties("stat2") - ); - Assert.assertTrue(stats.getDuties("stat3").isEmpty()); - - Assert.assertEquals(-4, stats.getDutyStat("stat1", "duty1")); - Assert.assertEquals(2, stats.getDutyStat("stat1", "duty2")); - Assert.assertEquals(1, stats.getDutyStat("stat2", "duty1")); - } - - @Test - public void testForEachDutyStat() - { - final Map expected = ImmutableMap.of( - "duty1", 1L, - "duty2", 2L, - "duty3", 3L - ); - final Map actual = new HashMap<>(); - - expected.forEach( - (duty, count) -> stats.addToDutyStat("stat", duty, count) - ); - - stats.forEachDutyStat("stat0", (duty, count) -> Assert.fail()); - stats.forEachDutyStat("stat", actual::put); - - Assert.assertEquals(expected, actual); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyBenchmark.java b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyBenchmark.java index e5ce98f7af6..2ba2142df23 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyBenchmark.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyBenchmark.java @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.balancer.CostBalancerStrategy; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.junit.Ignore; @@ -36,6 +37,7 @@ import org.junit.runners.Parameterized; import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.Random; import java.util.stream.Collectors; @@ -105,7 +107,8 @@ public class CostBalancerStrategyBenchmark extends AbstractBenchmark { DataSegment segment = DataSegment.builder().dataSource("testds").version("1000") .interval(interval1).size(100L).build(); - selected = strategy.findNewSegmentHomeReplicator(segment, serverHolderList); + Iterator candidates = strategy.findServersToLoadSegment(segment, serverHolderList); + selected = candidates.hasNext() ? candidates.next() : null; } // Benchmark Joda Interval Gap impl vs CostBalancer.gapMillis diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java index 9b1aa711311..e3c31f6471c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java @@ -28,6 +28,8 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Collections; @@ -39,10 +41,12 @@ import java.util.Objects; */ public class CreateDataSegments { + private static final DateTime DEFAULT_START = DateTimes.of("2012-10-24"); + private final String datasource; - private DateTime startTime; - private Granularity granularity; + private DateTime startTime = DEFAULT_START; + private Granularity granularity = Granularities.DAY; private int numPartitions = 1; private int numIntervals = 1; @@ -103,7 +107,7 @@ public class CreateDataSegments nextInterval, new NumberedShardSpec(numPartition, numPartitions), ++uniqueIdInInterval, - sizeMb + sizeMb << 20 ) ); } @@ -118,6 +122,7 @@ public class CreateDataSegments */ private static class NumberedDataSegment extends DataSegment { + private static final DateTimeFormatter FORMATTER = DateTimeFormat.forPattern("yyyyMMdd"); private final int uniqueId; private NumberedDataSegment( @@ -145,7 +150,9 @@ public class CreateDataSegments @Override public String toString() { - return "{" + getDataSource() + "::" + uniqueId + "}"; + return "{" + getDataSource() + + "::" + getInterval().getStart().toString(FORMATTER) + + "::" + uniqueId + "}"; } } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index 4e2f8bb2160..b9b6fc8516f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -24,7 +24,6 @@ import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; @@ -35,24 +34,24 @@ import org.apache.druid.client.CoordinatorServerView; import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidDataSource; -import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.curator.CuratorTestBase; import org.apache.druid.curator.CuratorUtils; -import org.apache.druid.curator.discovery.LatchableServiceAnnouncer; -import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.segment.TestHelper; -import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups; +import org.apache.druid.server.coordinator.balancer.BalancerStrategy; +import org.apache.druid.server.coordinator.loading.CuratorLoadQueuePeon; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.server.coordinator.loading.LoadQueueTaskMaster; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; +import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig; +import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.initialization.ZkPathsConfig; -import org.apache.druid.server.lookup.cache.LookupCoordinatorManager; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.testing.DeadlockDetectingTimeout; import org.apache.druid.timeline.DataSegment; @@ -68,36 +67,28 @@ import org.junit.Test; import org.junit.rules.TestRule; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; /** * This tests zookeeper specific coordinator/load queue/historical interactions, such as moving segments by the balancer */ public class CuratorDruidCoordinatorTest extends CuratorTestBase { - private DruidCoordinator coordinator; private SegmentsMetadataManager segmentsMetadataManager; private DataSourcesSnapshot dataSourcesSnapshot; private DruidCoordinatorRuntimeParams coordinatorRuntimeParams; - private ConcurrentMap loadManagementPeons; private LoadQueuePeon sourceLoadQueuePeon; private LoadQueuePeon destinationLoadQueuePeon; - private CountDownLatch leaderAnnouncerLatch; - private CountDownLatch leaderUnannouncerLatch; private PathChildrenCache sourceLoadQueueChildrenCache; private PathChildrenCache destinationLoadQueueChildrenCache; - private DruidCoordinatorConfig druidCoordinatorConfig; - private JacksonConfigManager configManager; private static final String SEGPATH = "/druid/segments"; private static final String SOURCE_LOAD_PATH = "/druid/loadQueue/localhost:1"; @@ -134,23 +125,6 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class); coordinatorRuntimeParams = EasyMock.createNiceMock(DruidCoordinatorRuntimeParams.class); - configManager = EasyMock.createNiceMock(JacksonConfigManager.class); - EasyMock.expect( - configManager.watch( - EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY), - EasyMock.anyObject(Class.class), - EasyMock.anyObject() - ) - ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); - EasyMock.expect( - configManager.watch( - EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), - EasyMock.anyObject(Class.class), - EasyMock.anyObject() - ) - ).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes(); - EasyMock.replay(configManager); - setupServerAndCurator(); curator.start(); curator.blockUntilConnected(); @@ -159,7 +133,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase curator.create().creatingParentsIfNeeded().forPath(DESTINATION_LOAD_PATH); final ObjectMapper objectMapper = new DefaultObjectMapper(); - druidCoordinatorConfig = new TestDruidCoordinatorConfig.Builder() + DruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig.Builder() .withCoordinatorStartDelay(new Duration(COORDINATOR_START_DELAY)) .withCoordinatorPeriod(new Duration(COORDINATOR_PERIOD)) .withCoordinatorKillPeriod(new Duration(COORDINATOR_PERIOD)) @@ -196,9 +170,6 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase callbackExec, druidCoordinatorConfig ); - loadManagementPeons = new ConcurrentHashMap<>(); - leaderAnnouncerLatch = new CountDownLatch(1); - leaderUnannouncerLatch = new CountDownLatch(1); } @After @@ -256,20 +227,14 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase setupZNodeForServer(source, zkPathsConfig, jsonMapper); setupZNodeForServer(dest, zkPathsConfig, jsonMapper); - final List sourceSegments = Lists.transform( - ImmutableList.of( - Pair.of("2011-04-01/2011-04-03", "v1"), - Pair.of("2011-04-03/2011-04-06", "v1"), - Pair.of("2011-04-06/2011-04-09", "v1") - ), - input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs) + final List sourceSegments = Arrays.asList( + createSegment("2011-04-01/2011-04-03", "v1"), + createSegment("2011-04-03/2011-04-06", "v1"), + createSegment("2011-04-06/2011-04-09", "v1") ); - final List destinationSegments = Lists.transform( - ImmutableList.of( - Pair.of("2011-03-31/2011-04-01", "v1") - ), - input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs) + final List destinationSegments = Collections.singletonList( + createSegment("2011-03-31/2011-04-01", "v1") ); DataSegment segmentToMove = sourceSegments.get(2); @@ -320,10 +285,8 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase Assert.assertTrue(timing.forWaiting().awaitLatch(srcCountdown)); Assert.assertTrue(timing.forWaiting().awaitLatch(destCountdown)); - - loadManagementPeons.put("localhost:1", sourceLoadQueuePeon); - loadManagementPeons.put("localhost:2", destinationLoadQueuePeon); - + sourceSegments.forEach(source::addDataSegment); + destinationSegments.forEach(dest::addDataSegment); segmentRemovedLatch = new CountDownLatch(1); segmentAddedLatch = new CountDownLatch(1); @@ -333,18 +296,50 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase EasyMock.replay(druidDataSource); EasyMock.expect(segmentsMetadataManager.getImmutableDataSourceWithUsedSegments(EasyMock.anyString())) .andReturn(druidDataSource); - EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()).andReturn(dataSourcesSnapshot).anyTimes(); - EasyMock.replay(segmentsMetadataManager, coordinatorRuntimeParams); + EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()) + .andReturn(dataSourcesSnapshot).anyTimes(); + final CoordinatorDynamicConfig dynamicConfig = + CoordinatorDynamicConfig.builder().withUseRoundRobinSegmentAssignment(false).build(); + EasyMock.expect(coordinatorRuntimeParams.getCoordinatorDynamicConfig()) + .andReturn(dynamicConfig) + .anyTimes(); + EasyMock.expect(coordinatorRuntimeParams.getSegmentLoadingConfig()) + .andReturn(SegmentLoadingConfig.create(dynamicConfig, 100)) + .anyTimes(); - EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes(); + final ServerHolder sourceServer = new ServerHolder(source.toImmutableDruidServer(), sourceLoadQueuePeon); + final ServerHolder destinationServer = new ServerHolder(dest.toImmutableDruidServer(), destinationLoadQueuePeon); + final DruidCluster cluster = DruidCluster.builder().add(sourceServer).add(destinationServer).build(); + + final BalancerStrategy balancerStrategy = EasyMock.mock(BalancerStrategy.class); + EasyMock.expect( + balancerStrategy.findDestinationServerToMoveSegment( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + ) + ).andReturn(destinationServer).atLeastOnce(); + EasyMock.expect(coordinatorRuntimeParams.getBalancerStrategy()) + .andReturn(balancerStrategy).anyTimes(); + EasyMock.expect(coordinatorRuntimeParams.getDruidCluster()).andReturn(cluster).anyTimes(); + EasyMock.replay(segmentsMetadataManager, coordinatorRuntimeParams, balancerStrategy); + + EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())) + .andReturn(druidDataSource).anyTimes(); EasyMock.replay(dataSourcesSnapshot); - coordinator.moveSegment( - coordinatorRuntimeParams, - source.toImmutableDruidServer(), - dest.toImmutableDruidServer(), - sourceSegments.get(2), - null + LoadQueueTaskMaster taskMaster = EasyMock.createMock(LoadQueueTaskMaster.class); + EasyMock.expect(taskMaster.isHttpLoading()).andReturn(false).anyTimes(); + EasyMock.replay(taskMaster); + + // Move the segment from source to dest + SegmentLoadQueueManager loadQueueManager = + new SegmentLoadQueueManager(baseView, segmentsMetadataManager, taskMaster); + StrategicSegmentAssigner segmentAssigner = createSegmentAssigner(loadQueueManager, coordinatorRuntimeParams); + segmentAssigner.moveSegment( + segmentToMove, + sourceServer, + Collections.singletonList(destinationServer) ); // wait for destination server to load segment @@ -365,45 +360,6 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase Assert.assertEquals(2, servers.get(1).getTotalSegments()); } - private static class TestDruidLeaderSelector implements DruidLeaderSelector - { - private volatile Listener listener; - private volatile String leader; - - @Override - public String getCurrentLeader() - { - return leader; - } - - @Override - public boolean isLeader() - { - return leader != null; - } - - @Override - public int localTerm() - { - return 0; - } - - @Override - public void registerListener(Listener listener) - { - this.listener = listener; - leader = "what:1234"; - listener.becomeLeader(); - } - - @Override - public void unregisterListener() - { - leader = null; - listener.stopBeingLeader(); - } - } - private void setupView() throws Exception { baseView = new BatchServerInventoryView( @@ -455,35 +411,9 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase sourceLoadQueuePeon.start(); destinationLoadQueuePeon.start(); - - final LoadQueueTaskMaster loadQueueTaskMaster = EasyMock.createMock(LoadQueueTaskMaster.class); - EasyMock.expect(loadQueueTaskMaster.isHttpLoading()).andReturn(false).anyTimes(); - EasyMock.replay(loadQueueTaskMaster); - - coordinator = new DruidCoordinator( - druidCoordinatorConfig, - configManager, - segmentsMetadataManager, - baseView, - EasyMock.createNiceMock(MetadataRuleManager.class), - new NoopServiceEmitter(), - (corePoolSize, nameFormat) -> Executors.newSingleThreadScheduledExecutor(), - null, - loadQueueTaskMaster, - new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), - new DruidNode("hey", "what", false, 1234, null, true, false), - loadManagementPeons, - null, - null, - new CoordinatorCustomDutyGroups(ImmutableSet.of()), - new CostBalancerStrategyFactory(), - EasyMock.createNiceMock(LookupCoordinatorManager.class), - new TestDruidLeaderSelector(), - null - ); } - private DataSegment dataSegmentWithIntervalAndVersion(String intervalStr, String version) + private DataSegment createSegment(String intervalStr, String version) { return DataSegment.builder() .dataSource("test_curator_druid_coordinator") @@ -497,4 +427,18 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase .size(0) .build(); } + + private StrategicSegmentAssigner createSegmentAssigner( + SegmentLoadQueueManager loadQueueManager, + DruidCoordinatorRuntimeParams params + ) + { + return new StrategicSegmentAssigner( + loadQueueManager, + params.getDruidCluster(), + params.getBalancerStrategy(), + params.getSegmentLoadingConfig(), + new CoordinatorRunStats() + ); + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java deleted file mode 100644 index 5fb100073ea..00000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java +++ /dev/null @@ -1,68 +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.druid.server.coordinator; - -import javax.annotation.Nullable; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -public final class DruidClusterBuilder -{ - public static DruidClusterBuilder newBuilder() - { - return new DruidClusterBuilder(); - } - - private @Nullable Set realtimes = null; - private final Map> historicals = new HashMap<>(); - private @Nullable Set brokers = null; - - private DruidClusterBuilder() - { - } - - public DruidClusterBuilder withRealtimes(ServerHolder... realtimes) - { - this.realtimes = new HashSet<>(Arrays.asList(realtimes)); - return this; - } - - public DruidClusterBuilder withBrokers(ServerHolder... brokers) - { - this.brokers = new HashSet<>(Arrays.asList(brokers)); - return this; - } - - public DruidClusterBuilder addTier(String tierName, ServerHolder... historicals) - { - if (this.historicals.putIfAbsent(tierName, Arrays.asList(historicals)) != null) { - throw new IllegalArgumentException("Duplicate tier: " + tierName); - } - return this; - } - - public DruidCluster build() - { - return DruidCluster.createDruidClusterFromBuilderInTest(realtimes, historicals, brokers); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java index b1e5dfdaedc..182d58dd387 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java @@ -19,21 +19,16 @@ package org.apache.druid.server.coordinator; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.client.ImmutableDruidDataSource; -import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.client.DruidServer; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NoneShardSpec; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.NavigableSet; @@ -42,100 +37,62 @@ import java.util.stream.Collectors; public class DruidClusterTest { - private static final List SEGMENTS = ImmutableList.of( - new DataSegment( - "test", - Intervals.of("2015-04-12/2015-04-13"), - "1", - ImmutableMap.of("containerName", "container1", "blobPath", "blobPath1"), - null, - null, - NoneShardSpec.instance(), - 0, - 1 - ), - new DataSegment( - "test", - Intervals.of("2015-04-12/2015-04-13"), - "1", - ImmutableMap.of("containerName", "container2", "blobPath", "blobPath2"), - null, - null, - NoneShardSpec.instance(), - 0, - 1 - ) - ); - - private static final Map DATA_SOURCES = ImmutableMap.of( - "src1", new ImmutableDruidDataSource("src1", Collections.emptyMap(), Collections.singletonList(SEGMENTS.get(0))), - "src2", new ImmutableDruidDataSource("src2", Collections.emptyMap(), Collections.singletonList(SEGMENTS.get(0))) - ); + private static final List SEGMENTS = CreateDataSegments + .ofDatasource("test") + .forIntervals(2, Granularities.DAY) + .startingAt("2015-04-12") + .withNumPartitions(1) + .eachOfSizeInMb(100); private static final ServerHolder NEW_REALTIME = new ServerHolder( - new ImmutableDruidServer( - new DruidServerMetadata("name1", "host2", null, 100L, ServerType.REALTIME, "tier1", 0), - 0L, - ImmutableMap.of("src1", DATA_SOURCES.get("src1")), - 1 - ), + new DruidServer("name1", "host2", null, 100L, ServerType.REALTIME, "tier1", 0) + .addDataSegment(SEGMENTS.get(0)).toImmutableDruidServer(), new LoadQueuePeonTester() ); private static final ServerHolder NEW_HISTORICAL = new ServerHolder( - new ImmutableDruidServer( - new DruidServerMetadata("name1", "host2", null, 100L, ServerType.HISTORICAL, "tier1", 0), - 0L, - ImmutableMap.of("src1", DATA_SOURCES.get("src1")), - 1 - ), + new DruidServer("name1", "host2", null, 100L, ServerType.HISTORICAL, "tier1", 0) + .addDataSegment(SEGMENTS.get(0)).toImmutableDruidServer(), new LoadQueuePeonTester() ); - private DruidCluster cluster; + private DruidCluster.Builder clusterBuilder; @Before public void setup() { - cluster = DruidClusterBuilder - .newBuilder() - .withRealtimes( + clusterBuilder = DruidCluster + .builder() + .add( new ServerHolder( - new ImmutableDruidServer( - new DruidServerMetadata("name1", "host1", null, 100L, ServerType.REALTIME, "tier1", 0), - 0L, - ImmutableMap.of("src1", DATA_SOURCES.get("src1")), - 1 - ), + new DruidServer("name1", "host1", null, 100L, ServerType.REALTIME, "tier1", 0) + .addDataSegment(SEGMENTS.get(0)).toImmutableDruidServer(), new LoadQueuePeonTester() ) ) - .addTier( - "tier1", + .add( new ServerHolder( - new ImmutableDruidServer( - new DruidServerMetadata("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0), - 0L, - ImmutableMap.of("src1", DATA_SOURCES.get("src1")), - 1 - ), + new DruidServer("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0) + .addDataSegment(SEGMENTS.get(0)).toImmutableDruidServer(), new LoadQueuePeonTester() ) - ) - .build(); + ); } @Test public void testAdd() { + DruidCluster cluster = clusterBuilder.build(); Assert.assertEquals(1, cluster.getHistoricals().values().stream().mapToInt(Collection::size).sum()); Assert.assertEquals(1, cluster.getRealtimes().size()); - cluster.add(NEW_REALTIME); + clusterBuilder.add(NEW_REALTIME); + cluster = clusterBuilder.build(); Assert.assertEquals(1, cluster.getHistoricals().values().stream().mapToInt(Collection::size).sum()); Assert.assertEquals(2, cluster.getRealtimes().size()); - cluster.add(NEW_HISTORICAL); + clusterBuilder.add(NEW_HISTORICAL); + cluster = clusterBuilder.build(); Assert.assertEquals(2, cluster.getHistoricals().values().stream().mapToInt(Collection::size).sum()); Assert.assertEquals(2, cluster.getRealtimes().size()); } @@ -143,8 +100,10 @@ public class DruidClusterTest @Test public void testGetAllServers() { - cluster.add(NEW_REALTIME); - cluster.add(NEW_HISTORICAL); + clusterBuilder.add(NEW_REALTIME); + clusterBuilder.add(NEW_HISTORICAL); + + DruidCluster cluster = clusterBuilder.build(); final Set expectedRealtimes = cluster.getRealtimes(); final Map> expectedHistoricals = cluster.getHistoricals(); @@ -153,7 +112,9 @@ public class DruidClusterTest Assert.assertTrue(allServers.containsAll(cluster.getRealtimes())); Assert.assertTrue( allServers.containsAll( - cluster.getHistoricals().values().stream().flatMap(Collection::stream).collect(Collectors.toList()) + cluster.getHistoricals().values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toList()) ) ); @@ -164,8 +125,8 @@ public class DruidClusterTest @Test public void testIsEmpty() { - final DruidCluster emptyCluster = new DruidCluster(); - Assert.assertFalse(cluster.isEmpty()); + final DruidCluster emptyCluster = DruidCluster.EMPTY; + Assert.assertFalse(clusterBuilder.build().isEmpty()); Assert.assertTrue(emptyCluster.isEmpty()); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 4076ce9d3a8..dd442405cd6 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -49,25 +49,25 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDuty; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroup; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups; import org.apache.druid.server.coordinator.duty.CoordinatorDuty; -import org.apache.druid.server.coordinator.duty.EmitClusterStatsAndMetrics; import org.apache.druid.server.coordinator.duty.KillSupervisorsCustomDuty; -import org.apache.druid.server.coordinator.duty.LogUsedSegments; import org.apache.druid.server.coordinator.duty.NewestSegmentFirstPolicy; +import org.apache.druid.server.coordinator.loading.CuratorLoadQueuePeon; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.server.coordinator.loading.LoadQueueTaskMaster; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.ForeverLoadRule; import org.apache.druid.server.coordinator.rules.IntervalLoadRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.server.lookup.cache.LookupCoordinatorManager; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.easymock.Capture; import org.easymock.EasyMock; import org.joda.time.Duration; import org.junit.After; @@ -80,8 +80,6 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; @@ -97,12 +95,10 @@ public class DruidCoordinatorTest extends CuratorTestBase private DruidCoordinator coordinator; private SegmentsMetadataManager segmentsMetadataManager; private DataSourcesSnapshot dataSourcesSnapshot; - private DruidCoordinatorRuntimeParams coordinatorRuntimeParams; private BatchServerInventoryView serverInventoryView; private ScheduledExecutorFactory scheduledExecutorFactory; private DruidServer druidServer; - private ConcurrentMap loadManagementPeons; private LoadQueuePeon loadQueuePeon; private LoadQueueTaskMaster loadQueueTaskMaster; private MetadataRuleManager metadataRuleManager; @@ -118,13 +114,13 @@ public class DruidCoordinatorTest extends CuratorTestBase @Before public void setUp() throws Exception { - druidServer = EasyMock.createMock(DruidServer.class); + druidServer = new DruidServer("from", "from", null, 5L, ServerType.HISTORICAL, "tier1", 0); serverInventoryView = EasyMock.createMock(BatchServerInventoryView.class); segmentsMetadataManager = EasyMock.createNiceMock(SegmentsMetadataManager.class); dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class); - coordinatorRuntimeParams = EasyMock.createNiceMock(DruidCoordinatorRuntimeParams.class); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); loadQueueTaskMaster = EasyMock.createMock(LoadQueueTaskMaster.class); + JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( configManager.watch( @@ -170,7 +166,6 @@ public class DruidCoordinatorTest extends CuratorTestBase ); loadQueuePeon.start(); druidNode = new DruidNode("hey", "what", false, 1234, null, true, false); - loadManagementPeons = new ConcurrentHashMap<>(); scheduledExecutorFactory = (corePoolSize, nameFormat) -> Executors.newSingleThreadScheduledExecutor(); leaderAnnouncerLatch = new CountDownLatch(1); leaderUnannouncerLatch = new CountDownLatch(1); @@ -184,11 +179,11 @@ public class DruidCoordinatorTest extends CuratorTestBase scheduledExecutorFactory, null, loadQueueTaskMaster, + new SegmentLoadQueueManager(serverInventoryView, segmentsMetadataManager, loadQueueTaskMaster), new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), druidNode, - loadManagementPeons, - null, new HashSet<>(), + null, new CoordinatorCustomDutyGroups(ImmutableSet.of()), new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), @@ -205,101 +200,6 @@ public class DruidCoordinatorTest extends CuratorTestBase tearDownServerAndCurator(); } - @Test - public void testMoveSegment() - { - final DataSegment segment = EasyMock.createNiceMock(DataSegment.class); - EasyMock.expect(segment.getId()).andReturn(SegmentId.dummy("dummySegment")); - EasyMock.expect(segment.getDataSource()).andReturn("dummyDataSource"); - EasyMock.replay(segment); - - loadQueuePeon = EasyMock.createNiceMock(LoadQueuePeon.class); - EasyMock.expect(loadQueuePeon.getLoadQueueSize()).andReturn(new Long(1)); - loadQueuePeon.markSegmentToDrop(segment); - EasyMock.expectLastCall().once(); - Capture loadCallbackCapture = Capture.newInstance(); - Capture dropCallbackCapture = Capture.newInstance(); - loadQueuePeon.loadSegment(EasyMock.anyObject(DataSegment.class), EasyMock.capture(loadCallbackCapture)); - EasyMock.expectLastCall().once(); - loadQueuePeon.dropSegment(EasyMock.anyObject(DataSegment.class), EasyMock.capture(dropCallbackCapture)); - EasyMock.expectLastCall().once(); - loadQueuePeon.unmarkSegmentToDrop(segment); - EasyMock.expectLastCall().once(); - EasyMock.expect(loadQueuePeon.getSegmentsToDrop()).andReturn(new HashSet<>()).once(); - EasyMock.replay(loadQueuePeon); - - ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); - EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(segment); - EasyMock.replay(druidDataSource); - EasyMock - .expect(segmentsMetadataManager.getImmutableDataSourceWithUsedSegments(EasyMock.anyString())) - .andReturn(druidDataSource); - EasyMock.replay(segmentsMetadataManager); - EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes(); - EasyMock.replay(dataSourcesSnapshot); - scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); - EasyMock.replay(scheduledExecutorFactory); - EasyMock.replay(metadataRuleManager); - ImmutableDruidDataSource dataSource = EasyMock.createMock(ImmutableDruidDataSource.class); - EasyMock.expect(dataSource.getSegments()).andReturn(Collections.singletonList(segment)).anyTimes(); - EasyMock.replay(dataSource); - EasyMock.expect(druidServer.toImmutableDruidServer()).andReturn( - new ImmutableDruidServer( - new DruidServerMetadata("from", null, null, 5L, ServerType.HISTORICAL, null, 0), - 1L, - ImmutableMap.of("dummyDataSource", dataSource), - 1 - ) - ).atLeastOnce(); - EasyMock.replay(druidServer); - - DruidServer druidServer2 = EasyMock.createMock(DruidServer.class); - - EasyMock.expect(druidServer2.toImmutableDruidServer()).andReturn( - new ImmutableDruidServer( - new DruidServerMetadata("to", null, null, 5L, ServerType.HISTORICAL, null, 0), - 1L, - ImmutableMap.of("dummyDataSource", dataSource), - 1 - ) - ).atLeastOnce(); - EasyMock.replay(druidServer2); - - loadManagementPeons.put("from", loadQueuePeon); - loadManagementPeons.put("to", loadQueuePeon); - - EasyMock.expect(serverInventoryView.isSegmentLoadedByServer("to", segment)).andReturn(true).once(); - EasyMock.replay(serverInventoryView); - - EasyMock.expect(loadQueueTaskMaster.isHttpLoading()).andReturn(false).anyTimes(); - EasyMock.replay(loadQueueTaskMaster); - - mockCoordinatorRuntimeParams(); - - coordinator.moveSegment( - coordinatorRuntimeParams, - druidServer.toImmutableDruidServer(), - druidServer2.toImmutableDruidServer(), - segment, - null - ); - - LoadPeonCallback loadCallback = loadCallbackCapture.getValue(); - loadCallback.execute(true); - - LoadPeonCallback dropCallback = dropCallbackCapture.getValue(); - dropCallback.execute(true); - - EasyMock.verify(druidServer, druidServer2, loadQueuePeon, serverInventoryView, metadataRuleManager); - EasyMock.verify(coordinatorRuntimeParams); - } - - private void mockCoordinatorRuntimeParams() - { - EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()).andReturn(this.dataSourcesSnapshot).anyTimes(); - EasyMock.replay(coordinatorRuntimeParams); - } - @Test(timeout = 60_000L) public void testCoordinatorRun() throws Exception { @@ -310,8 +210,6 @@ public class DruidCoordinatorTest extends CuratorTestBase Rule foreverLoadRule = new ForeverLoadRule(ImmutableMap.of(tier, 2)); EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString())) .andReturn(ImmutableList.of(foreverLoadRule)).atLeastOnce(); - EasyMock.expect(metadataRuleManager.getAllRules()) - .andReturn(ImmutableMap.of(dataSource, ImmutableList.of(foreverLoadRule))).atLeastOnce(); metadataRuleManager.stop(); EasyMock.expectLastCall().once(); @@ -343,12 +241,12 @@ public class DruidCoordinatorTest extends CuratorTestBase // Setup ServerInventoryView druidServer = new DruidServer("server1", "localhost", null, 5L, ServerType.HISTORICAL, tier, 0); - loadManagementPeons.put("server1", loadQueuePeon); + setupPeons(Collections.singletonMap("server1", loadQueuePeon)); EasyMock.expect(serverInventoryView.getInventory()).andReturn( ImmutableList.of(druidServer) ).atLeastOnce(); EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes(); - EasyMock.replay(serverInventoryView); + EasyMock.replay(serverInventoryView, loadQueueTaskMaster); coordinator.start(); // Wait for this coordinator to become leader @@ -371,15 +269,15 @@ public class DruidCoordinatorTest extends CuratorTestBase serviceEmitter.latch = coordinatorRunLatch; coordinatorRunLatch.await(); - Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus()); + Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getDatasourceToLoadStatus()); Object2IntMap numsUnavailableUsedSegmentsPerDataSource = - coordinator.computeNumsUnavailableUsedSegmentsPerDataSource(); + coordinator.getDatasourceToUnavailableSegmentCount(); Assert.assertEquals(1, numsUnavailableUsedSegmentsPerDataSource.size()); Assert.assertEquals(0, numsUnavailableUsedSegmentsPerDataSource.getInt(dataSource)); Map> underReplicationCountsPerDataSourcePerTier = - coordinator.computeUnderReplicationCountsPerDataSourcePerTier(); + coordinator.getTierToDatasourceToUnderReplicatedCount(false); Assert.assertNotNull(underReplicationCountsPerDataSourcePerTier); Assert.assertEquals(1, underReplicationCountsPerDataSourcePerTier.size()); @@ -392,9 +290,8 @@ public class DruidCoordinatorTest extends CuratorTestBase // The load rules asks for 2 replicas, therefore 1 replica should still be pending Assert.assertEquals(1L, underRepliicationCountsPerDataSource.getLong(dataSource)); - Map> underReplicationCountsPerDataSourcePerTierUsingClusterView = - coordinator.computeUnderReplicationCountsPerDataSourcePerTierUsingClusterView(); + coordinator.getTierToDatasourceToUnderReplicatedCount(true); Assert.assertNotNull(underReplicationCountsPerDataSourcePerTier); Assert.assertEquals(1, underReplicationCountsPerDataSourcePerTier.size()); @@ -454,7 +351,7 @@ public class DruidCoordinatorTest extends CuratorTestBase true, Execs.singleThreaded("coordinator_test_path_children_cache_cold-%d") ); - loadManagementPeons.putAll(ImmutableMap.of("hot", loadQueuePeon, "cold", loadQueuePeonCold)); + setupPeons(ImmutableMap.of("hot", loadQueuePeon, "cold", loadQueuePeonCold)); loadQueuePeonCold.start(); pathChildrenCache.start(); @@ -467,15 +364,13 @@ public class DruidCoordinatorTest extends CuratorTestBase EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString())) .andReturn(ImmutableList.of(hotTier, coldTier)).atLeastOnce(); - EasyMock.expect(metadataRuleManager.getAllRules()) - .andReturn(ImmutableMap.of(dataSource, ImmutableList.of(hotTier, coldTier))).atLeastOnce(); EasyMock.expect(serverInventoryView.getInventory()) .andReturn(ImmutableList.of(hotServer, coldServer)) .atLeastOnce(); EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes(); - EasyMock.replay(metadataRuleManager, serverInventoryView); + EasyMock.replay(metadataRuleManager, serverInventoryView, loadQueueTaskMaster); coordinator.start(); leaderAnnouncerLatch.await(); // Wait for this coordinator to become leader @@ -489,16 +384,16 @@ public class DruidCoordinatorTest extends CuratorTestBase serviceEmitter.latch = coordinatorRunLatch; coordinatorRunLatch.await(); - Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus()); + Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getDatasourceToLoadStatus()); Map> underReplicationCountsPerDataSourcePerTier = - coordinator.computeUnderReplicationCountsPerDataSourcePerTier(); + coordinator.getTierToDatasourceToUnderReplicatedCount(false); Assert.assertEquals(2, underReplicationCountsPerDataSourcePerTier.size()); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(hotTierName).getLong(dataSource)); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(coldTierName).getLong(dataSource)); Map> underReplicationCountsPerDataSourcePerTierUsingClusterView = - coordinator.computeUnderReplicationCountsPerDataSourcePerTierUsingClusterView(); + coordinator.getTierToDatasourceToUnderReplicatedCount(true); Assert.assertEquals(2, underReplicationCountsPerDataSourcePerTierUsingClusterView.size()); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(hotTierName).getLong(dataSource)); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(coldTierName).getLong(dataSource)); @@ -519,7 +414,6 @@ public class DruidCoordinatorTest extends CuratorTestBase final String coldTierName = "cold"; final String tierName1 = "tier1"; final String tierName2 = "tier2"; - final Rule broadcastDistributionRule = new ForeverBroadcastDistributionRule(); final String loadPathCold = "/druid/loadqueue/cold:1234"; final String loadPathBroker1 = "/druid/loadqueue/broker1:1234"; final String loadPathBroker2 = "/druid/loadqueue/broker2:1234"; @@ -603,11 +497,13 @@ public class DruidCoordinatorTest extends CuratorTestBase Execs.singleThreaded("coordinator_test_path_children_cache_peon-%d") ); - loadManagementPeons.putAll(ImmutableMap.of("hot", loadQueuePeon, - "cold", loadQueuePeonCold, - "broker1", loadQueuePeonBroker1, - "broker2", loadQueuePeonBroker2, - "peon", loadQueuePeonPoenServer)); + setupPeons(ImmutableMap.of( + "hot", loadQueuePeon, + "cold", loadQueuePeonCold, + "broker1", loadQueuePeonBroker1, + "broker2", loadQueuePeonBroker2, + "peon", loadQueuePeonPoenServer + )); loadQueuePeonCold.start(); loadQueuePeonBroker1.start(); @@ -619,22 +515,21 @@ public class DruidCoordinatorTest extends CuratorTestBase pathChildrenCacheBroker2.start(); pathChildrenCachePeon.start(); - DruidDataSource[] druidDataSources = {new DruidDataSource(dataSource, Collections.emptyMap())}; - dataSegments.values().forEach(druidDataSources[0]::addSegment); + DruidDataSource druidDataSource = new DruidDataSource(dataSource, Collections.emptyMap()); + dataSegments.values().forEach(druidDataSource::addSegment); - setupSegmentsMetadataMock(druidDataSources[0]); + setupSegmentsMetadataMock(druidDataSource); + final Rule broadcastDistributionRule = new ForeverBroadcastDistributionRule(); EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString())) .andReturn(ImmutableList.of(broadcastDistributionRule)).atLeastOnce(); - EasyMock.expect(metadataRuleManager.getAllRules()) - .andReturn(ImmutableMap.of(dataSource, ImmutableList.of(broadcastDistributionRule))).atLeastOnce(); EasyMock.expect(serverInventoryView.getInventory()) .andReturn(ImmutableList.of(hotServer, coldServer, brokerServer1, brokerServer2, peonServer)) .atLeastOnce(); EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes(); - EasyMock.replay(metadataRuleManager, serverInventoryView); + EasyMock.replay(metadataRuleManager, serverInventoryView, loadQueueTaskMaster); coordinator.start(); leaderAnnouncerLatch.await(); // Wait for this coordinator to become leader @@ -654,10 +549,11 @@ public class DruidCoordinatorTest extends CuratorTestBase serviceEmitter.latch = coordinatorRunLatch; coordinatorRunLatch.await(); - Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus()); + Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getDatasourceToLoadStatus()); + // Under-replicated counts are updated only after the next coordinator run Map> underReplicationCountsPerDataSourcePerTier = - coordinator.computeUnderReplicationCountsPerDataSourcePerTier(); + coordinator.getTierToDatasourceToUnderReplicatedCount(false); Assert.assertEquals(4, underReplicationCountsPerDataSourcePerTier.size()); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(hotTierName).getLong(dataSource)); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(coldTierName).getLong(dataSource)); @@ -665,7 +561,7 @@ public class DruidCoordinatorTest extends CuratorTestBase Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(tierName2).getLong(dataSource)); Map> underReplicationCountsPerDataSourcePerTierUsingClusterView = - coordinator.computeUnderReplicationCountsPerDataSourcePerTierUsingClusterView(); + coordinator.getTierToDatasourceToUnderReplicatedCount(true); Assert.assertEquals(4, underReplicationCountsPerDataSourcePerTierUsingClusterView.size()); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(hotTierName).getLong(dataSource)); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(coldTierName).getLong(dataSource)); @@ -694,7 +590,7 @@ public class DruidCoordinatorTest extends CuratorTestBase EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference(dynamicConfig)).anyTimes(); + ).andReturn(new AtomicReference<>(dynamicConfig)).anyTimes(); ScheduledExecutorFactory scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); EasyMock.replay(configManager, dynamicConfig, scheduledExecutorFactory); @@ -708,6 +604,7 @@ public class DruidCoordinatorTest extends CuratorTestBase null, scheduledExecutorFactory, null, + loadQueueTaskMaster, null, null, null, @@ -720,32 +617,30 @@ public class DruidCoordinatorTest extends CuratorTestBase null ); - DruidCoordinator.DutiesRunnable duty = c.new DutiesRunnable(Collections.emptyList(), 0, "TEST"); // before initialization Assert.assertEquals(0, c.getCachedBalancerThreadNumber()); Assert.assertNull(c.getBalancerExec()); // first initialization - duty.initBalancerExecutor(); - System.out.println("c.getCachedBalancerThreadNumber(): " + c.getCachedBalancerThreadNumber()); + c.initBalancerExecutor(); Assert.assertEquals(5, c.getCachedBalancerThreadNumber()); ListeningExecutorService firstExec = c.getBalancerExec(); Assert.assertNotNull(firstExec); // second initialization, expect no changes as cachedBalancerThreadNumber is not changed - duty.initBalancerExecutor(); + c.initBalancerExecutor(); Assert.assertEquals(5, c.getCachedBalancerThreadNumber()); ListeningExecutorService secondExec = c.getBalancerExec(); Assert.assertNotNull(secondExec); - Assert.assertTrue(firstExec == secondExec); + Assert.assertSame(firstExec, secondExec); // third initialization, expect executor recreated as cachedBalancerThreadNumber is changed to 10 - duty.initBalancerExecutor(); + c.initBalancerExecutor(); Assert.assertEquals(10, c.getCachedBalancerThreadNumber()); ListeningExecutorService thirdExec = c.getBalancerExec(); Assert.assertNotNull(thirdExec); - Assert.assertFalse(secondExec == thirdExec); - Assert.assertFalse(firstExec == thirdExec); + Assert.assertNotSame(secondExec, thirdExec); + Assert.assertNotSame(firstExec, thirdExec); } @Test @@ -761,12 +656,12 @@ public class DruidCoordinatorTest extends CuratorTestBase serviceEmitter, scheduledExecutorFactory, null, + loadQueueTaskMaster, null, new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), druidNode, - loadManagementPeons, - ImmutableSet.of(), new HashSet<>(), + ImmutableSet.of(), emptyCustomDutyGroups, new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), @@ -784,7 +679,6 @@ public class DruidCoordinatorTest extends CuratorTestBase // CompactSegments returned by this method should be created using the DruidCoordinatorConfig in the DruidCoordinator CompactSegments duty = coordinator.initializeCompactSegmentsDuty(newestSegmentFirstPolicy); Assert.assertNotNull(duty); - Assert.assertEquals(druidCoordinatorConfig.getCompactionSkipLockedIntervals(), duty.isSkipLockedIntervals()); } @Test @@ -801,12 +695,12 @@ public class DruidCoordinatorTest extends CuratorTestBase serviceEmitter, scheduledExecutorFactory, null, + loadQueueTaskMaster, null, new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), druidNode, - loadManagementPeons, - ImmutableSet.of(), new HashSet<>(), + ImmutableSet.of(), customDutyGroups, new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), @@ -824,7 +718,6 @@ public class DruidCoordinatorTest extends CuratorTestBase // CompactSegments returned by this method should be created using the DruidCoordinatorConfig in the DruidCoordinator CompactSegments duty = coordinator.initializeCompactSegmentsDuty(newestSegmentFirstPolicy); Assert.assertNotNull(duty); - Assert.assertEquals(druidCoordinatorConfig.getCompactionSkipLockedIntervals(), duty.isSkipLockedIntervals()); } @Test @@ -849,12 +742,12 @@ public class DruidCoordinatorTest extends CuratorTestBase serviceEmitter, scheduledExecutorFactory, null, + loadQueueTaskMaster, null, new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), druidNode, - loadManagementPeons, - ImmutableSet.of(), new HashSet<>(), + ImmutableSet.of(), customDutyGroups, new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), @@ -870,7 +763,6 @@ public class DruidCoordinatorTest extends CuratorTestBase Assert.assertFalse(compactSegmentsDutyFromCustomGroups.isEmpty()); Assert.assertEquals(1, compactSegmentsDutyFromCustomGroups.size()); Assert.assertNotNull(compactSegmentsDutyFromCustomGroups.get(0)); - Assert.assertTrue(compactSegmentsDutyFromCustomGroups.get(0) instanceof CompactSegments); // CompactSegments returned by this method should be from the Custom Duty Group CompactSegments duty = coordinator.initializeCompactSegmentsDuty(newestSegmentFirstPolicy); @@ -891,16 +783,15 @@ public class DruidCoordinatorTest extends CuratorTestBase EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference(CoordinatorDynamicConfig.builder().build())).anyTimes(); + ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); EasyMock.expect( configManager.watch( EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference(CoordinatorCompactionConfig.empty())).anyTimes(); + ).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes(); EasyMock.replay(configManager); - EasyMock.expect(segmentsMetadataManager.isPollingDatabasePeriodically()).andReturn(true).anyTimes(); DruidDataSource dataSource = new DruidDataSource("dataSource1", Collections.emptyMap()); DataSegment dataSegment = new DataSegment( "dataSource1", @@ -920,33 +811,37 @@ public class DruidCoordinatorTest extends CuratorTestBase .expect(segmentsMetadataManager.getSnapshotOfDataSourcesWithAllUsedSegments()) .andReturn(dataSourcesSnapshot) .anyTimes(); + EasyMock.expect(segmentsMetadataManager.isPollingDatabasePeriodically()).andReturn(true).anyTimes(); + EasyMock.expect(segmentsMetadataManager.iterateAllUsedSegments()) + .andReturn(Collections.singletonList(dataSegment)).anyTimes(); EasyMock.replay(segmentsMetadataManager); EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes(); + EasyMock.expect(serverInventoryView.getInventory()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.replay(serverInventoryView); // Create CoordinatorCustomDutyGroups // We will have two groups and each group has one duty CountDownLatch latch1 = new CountDownLatch(1); - CoordinatorCustomDuty duty1 = new CoordinatorCustomDuty() { - @Override - public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) - { - latch1.countDown(); - return params; - } + CoordinatorCustomDuty duty1 = params -> { + latch1.countDown(); + return params; }; - CoordinatorCustomDutyGroup group1 = new CoordinatorCustomDutyGroup("group1", Duration.standardSeconds(1), ImmutableList.of(duty1)); + CoordinatorCustomDutyGroup group1 = new CoordinatorCustomDutyGroup( + "group1", + Duration.standardSeconds(1), + ImmutableList.of(duty1) + ); CountDownLatch latch2 = new CountDownLatch(1); - CoordinatorCustomDuty duty2 = new CoordinatorCustomDuty() { - @Override - public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) - { - latch2.countDown(); - return params; - } + CoordinatorCustomDuty duty2 = params -> { + latch2.countDown(); + return params; }; - CoordinatorCustomDutyGroup group2 = new CoordinatorCustomDutyGroup("group2", Duration.standardSeconds(1), ImmutableList.of(duty2)); + CoordinatorCustomDutyGroup group2 = new CoordinatorCustomDutyGroup( + "group2", + Duration.standardSeconds(1), + ImmutableList.of(duty2) + ); CoordinatorCustomDutyGroups groups = new CoordinatorCustomDutyGroups(ImmutableSet.of(group1, group2)); coordinator = new DruidCoordinator( @@ -958,12 +853,12 @@ public class DruidCoordinatorTest extends CuratorTestBase serviceEmitter, scheduledExecutorFactory, null, - null, + loadQueueTaskMaster, + new SegmentLoadQueueManager(serverInventoryView, segmentsMetadataManager, loadQueueTaskMaster), new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), druidNode, - loadManagementPeons, - null, new HashSet<>(), + null, groups, new CostBalancerStrategyFactory(), EasyMock.createNiceMock(LookupCoordinatorManager.class), @@ -978,38 +873,12 @@ public class DruidCoordinatorTest extends CuratorTestBase latch2.await(); } - @Test - public void testEmitClusterStatsAndMetricsAddedWhenNotInDutyList() - { - DruidCoordinator.DutiesRunnable dutyRunnable = coordinator.new DutiesRunnable(ImmutableList.of(new LogUsedSegments()), 0, "TEST"); - List duties = dutyRunnable.getDuties(); - int emitDutyFound = 0; - for (CoordinatorDuty duty : duties) { - if (duty instanceof EmitClusterStatsAndMetrics) { - emitDutyFound++; - } - } - Assert.assertEquals(1, emitDutyFound); - } - - @Test - public void testEmitClusterStatsAndMetricsNotAddedAgainWhenInDutyList() - { - DruidCoordinator.DutiesRunnable dutyRunnable = coordinator.new DutiesRunnable(ImmutableList.of(new LogUsedSegments(), new EmitClusterStatsAndMetrics(coordinator, "TEST", false)), 0, "TEST"); - List duties = dutyRunnable.getDuties(); - int emitDutyFound = 0; - for (CoordinatorDuty duty : duties) { - if (duty instanceof EmitClusterStatsAndMetrics) { - emitDutyFound++; - } - } - Assert.assertEquals(1, emitDutyFound); - } - - private CountDownLatch createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(int latchCount, - PathChildrenCache pathChildrenCache, - Map segments, - DruidServer server) + private CountDownLatch createCountDownLatchAndSetPathChildrenCacheListenerWithLatch( + int latchCount, + PathChildrenCache pathChildrenCache, + Map segments, + DruidServer server + ) { final CountDownLatch countDownLatch = new CountDownLatch(latchCount); pathChildrenCache.getListenable().addListener( @@ -1080,6 +949,13 @@ public class DruidCoordinatorTest extends CuratorTestBase .orElse(null); } + private void setupPeons(Map peonMap) + { + EasyMock.expect(loadQueueTaskMaster.giveMePeon(EasyMock.anyObject())).andAnswer( + () -> peonMap.get(((ImmutableDruidServer) EasyMock.getCurrentArgument(0)).getName()) + ).anyTimes(); + } + private static class TestDruidLeaderSelector implements DruidLeaderSelector { private volatile Listener listener; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java deleted file mode 100644 index aed032b196d..00000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/HttpLoadQueuePeonTest.java +++ /dev/null @@ -1,203 +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.druid.server.coordinator; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.RE; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.HttpResponseHandler; -import org.apache.druid.server.ServerTestHelper; -import org.apache.druid.server.coordination.DataSegmentChangeRequest; -import org.apache.druid.server.coordination.SegmentLoadDropHandler; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.jboss.netty.buffer.ChannelBuffers; -import org.jboss.netty.handler.codec.http.DefaultHttpResponse; -import org.jboss.netty.handler.codec.http.HttpResponse; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.jboss.netty.handler.codec.http.HttpVersion; -import org.joda.time.Duration; -import org.junit.Test; - -import java.io.ByteArrayInputStream; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; - -/** - */ -public class HttpLoadQueuePeonTest -{ - final DataSegment segment1 = new DataSegment( - "test1", Intervals.of("2014/2015"), "v1", - null, null, null, null, 0, 0 - ); - - final DataSegment segment2 = new DataSegment( - "test2", Intervals.of("2014/2015"), "v1", - null, null, null, null, 0, 0 - ); - - final DataSegment segment3 = new DataSegment( - "test3", Intervals.of("2014/2015"), "v1", - null, null, null, null, 0, 0 - ); - - final DataSegment segment4 = new DataSegment( - "test4", Intervals.of("2014/2015"), "v1", - null, null, null, null, 0, 0 - ); - - final TestDruidCoordinatorConfig config = new TestDruidCoordinatorConfig.Builder() - .withCoordinatorKillMaxSegments(10) - .withCoordinatorKillIgnoreDurationToRetain(false) - .withHttpLoadQueuePeonBatchSize(2) - .build(); - - @Test(timeout = 60_000L) - public void testSimple() throws Exception - { - HttpLoadQueuePeon httpLoadQueuePeon = new HttpLoadQueuePeon( - "http://dummy:4000", - ServerTestHelper.MAPPER, - new TestHttpClient(), - config, - Executors.newScheduledThreadPool( - 2, - Execs.makeThreadFactory("HttpLoadQueuePeonTest-%s") - ), - Execs.singleThreaded("HttpLoadQueuePeonTest") - ); - - httpLoadQueuePeon.start(); - - Map latches = ImmutableMap.of( - segment1.getId(), new CountDownLatch(1), - segment2.getId(), new CountDownLatch(1), - segment3.getId(), new CountDownLatch(1), - segment4.getId(), new CountDownLatch(1) - ); - - httpLoadQueuePeon.dropSegment(segment1, success -> latches.get(segment1.getId()).countDown()); - httpLoadQueuePeon.loadSegment(segment2, success -> latches.get(segment2.getId()).countDown()); - httpLoadQueuePeon.dropSegment(segment3, success -> latches.get(segment3.getId()).countDown()); - httpLoadQueuePeon.loadSegment(segment4, success -> latches.get(segment4.getId()).countDown()); - - latches.get(segment1.getId()).await(); - latches.get(segment2.getId()).await(); - latches.get(segment3.getId()).await(); - latches.get(segment4.getId()).await(); - - httpLoadQueuePeon.stop(); - } - - @Test(timeout = 60_000L) - public void testLoadDropAfterStop() throws Exception - { - HttpLoadQueuePeon httpLoadQueuePeon = new HttpLoadQueuePeon( - "http://dummy:4000", - ServerTestHelper.MAPPER, - new TestHttpClient(), - config, - Executors.newScheduledThreadPool( - 2, - Execs.makeThreadFactory("HttpLoadQueuePeonTest-%s") - ), - Execs.singleThreaded("HttpLoadQueuePeonTest") - ); - - httpLoadQueuePeon.start(); - - Map latches = ImmutableMap.of( - segment1.getId(), new CountDownLatch(1), - segment2.getId(), new CountDownLatch(1), - segment3.getId(), new CountDownLatch(1), - segment4.getId(), new CountDownLatch(1) - ); - - httpLoadQueuePeon.dropSegment(segment1, success -> latches.get(segment1.getId()).countDown()); - httpLoadQueuePeon.loadSegment(segment2, success -> latches.get(segment2.getId()).countDown()); - latches.get(segment1.getId()).await(); - latches.get(segment2.getId()).await(); - httpLoadQueuePeon.stop(); - httpLoadQueuePeon.dropSegment(segment3, success -> latches.get(segment3.getId()).countDown()); - httpLoadQueuePeon.loadSegment(segment4, success -> latches.get(segment4.getId()).countDown()); - latches.get(segment3.getId()).await(); - latches.get(segment4.getId()).await(); - - } - - private static class TestHttpClient implements HttpClient - { - @Override - public ListenableFuture go( - Request request, - HttpResponseHandler httpResponseHandler - ) - { - throw new UnsupportedOperationException("Not Implemented."); - } - - @Override - public ListenableFuture go( - Request request, - HttpResponseHandler httpResponseHandler, - Duration duration - ) - { - HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - httpResponse.setContent(ChannelBuffers.buffer(0)); - httpResponseHandler.handleResponse(httpResponse, null); - try { - List changeRequests = ServerTestHelper.MAPPER.readValue( - request.getContent().array(), new TypeReference>() - { - } - ); - - List statuses = new ArrayList<>(changeRequests.size()); - for (DataSegmentChangeRequest cr : changeRequests) { - statuses.add(new SegmentLoadDropHandler.DataSegmentChangeRequestAndStatus( - cr, - SegmentLoadDropHandler.Status.SUCCESS - )); - } - return (ListenableFuture) Futures.immediateFuture( - new ByteArrayInputStream( - ServerTestHelper.MAPPER - .writerWithType(HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF) - .writeValueAsBytes(statuses) - ) - ); - } - catch (Exception ex) { - throw new RE(ex, "Unexpected exception."); - } - } - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/RoundRobinServerSelectorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/RoundRobinServerSelectorTest.java index bcef7ca8a4d..4164bba3649 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/RoundRobinServerSelectorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/RoundRobinServerSelectorTest.java @@ -23,6 +23,8 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.IndexIO; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; +import org.apache.druid.server.coordinator.loading.RoundRobinServerSelector; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.junit.Assert; @@ -51,14 +53,14 @@ public class RoundRobinServerSelectorTest public void testSingleIterator() { final ServerHolder serverXL = createHistorical("serverXL", 1000); - final ServerHolder serverL = createHistorical("serverXL", 900); - final ServerHolder serverM = createHistorical("serverXL", 800); + final ServerHolder serverL = createHistorical("serverL", 900); + final ServerHolder serverM = createHistorical("serverM", 800); // This server is too small to house the segment - final ServerHolder serverXS = createHistorical("serverXL", 10); + final ServerHolder serverXS = createHistorical("serverXS", 10); - DruidCluster cluster = DruidClusterBuilder - .newBuilder() + DruidCluster cluster = DruidCluster + .builder() .addTier(TIER, serverXL, serverM, serverXS, serverL) .build(); final RoundRobinServerSelector selector = new RoundRobinServerSelector(cluster); @@ -78,14 +80,14 @@ public class RoundRobinServerSelectorTest public void testNextIteratorContinuesFromSamePosition() { final ServerHolder serverXL = createHistorical("serverXL", 1000); - final ServerHolder serverL = createHistorical("serverXL", 900); - final ServerHolder serverM = createHistorical("serverXL", 800); + final ServerHolder serverL = createHistorical("serverL", 900); + final ServerHolder serverM = createHistorical("serverM", 800); // This server is too small to house the segment - final ServerHolder serverXS = createHistorical("serverXL", 10); + final ServerHolder serverXS = createHistorical("serverXS", 10); - DruidCluster cluster = DruidClusterBuilder - .newBuilder() + DruidCluster cluster = DruidCluster + .builder() .addTier(TIER, serverXL, serverM, serverXS, serverL) .build(); final RoundRobinServerSelector selector = new RoundRobinServerSelector(cluster); @@ -109,10 +111,7 @@ public class RoundRobinServerSelectorTest @Test public void testNoServersInTier() { - DruidCluster cluster = DruidClusterBuilder - .newBuilder() - .addTier(TIER) - .build(); + DruidCluster cluster = DruidCluster.builder().addTier(TIER).build(); final RoundRobinServerSelector selector = new RoundRobinServerSelector(cluster); Iterator eligibleServers = selector.getServersInTierToLoadSegment(TIER, segment); @@ -122,16 +121,13 @@ public class RoundRobinServerSelectorTest @Test public void testNoEligibleServerInTier() { - DruidCluster cluster = DruidClusterBuilder - .newBuilder() - .addTier( - TIER, - createHistorical("server1", 40), - createHistorical("server2", 30), - createHistorical("server3", 10), - createHistorical("server4", 20) - ) - .build(); + DruidCluster cluster = DruidCluster.builder().addTier( + TIER, + createHistorical("server1", 40), + createHistorical("server2", 30), + createHistorical("server3", 10), + createHistorical("server4", 20) + ).build(); final RoundRobinServerSelector selector = new RoundRobinServerSelector(cluster); // Verify that only eligible servers are returned in order of available size @@ -142,7 +138,8 @@ public class RoundRobinServerSelectorTest private ServerHolder createHistorical(String name, long size) { return new ServerHolder( - new DruidServer(name, name, null, size, ServerType.HISTORICAL, TIER, 1).toImmutableDruidServer(), + new DruidServer(name, name, null, size, ServerType.HISTORICAL, TIER, 1) + .toImmutableDruidServer(), new LoadQueuePeonTester() ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/RunRulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/RunRulesTest.java deleted file mode 100644 index 3256a0ad766..00000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/RunRulesTest.java +++ /dev/null @@ -1,1480 +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.druid.server.coordinator; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.druid.client.DruidServer; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.emitter.service.ServiceEventBuilder; -import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.metadata.SegmentsMetadataManager; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.duty.RunRules; -import org.apache.druid.server.coordinator.rules.ForeverLoadRule; -import org.apache.druid.server.coordinator.rules.IntervalDropRule; -import org.apache.druid.server.coordinator.rules.IntervalLoadRule; -import org.apache.druid.server.coordinator.rules.LoadRule; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NoneShardSpec; -import org.easymock.EasyMock; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -/** - */ -public class RunRulesTest -{ - public static final CoordinatorDynamicConfig COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS = - CoordinatorDynamicConfig.builder().withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L).build(); - - private DruidCoordinator coordinator; - private LoadQueuePeon mockPeon; - private List usedSegments; - private RunRules ruleRunner; - private ServiceEmitter emitter; - private MetadataRuleManager databaseRuleManager; - private SegmentsMetadataManager segmentsMetadataManager; - - @Before - public void setUp() - { - coordinator = EasyMock.createMock(DruidCoordinator.class); - mockPeon = EasyMock.createMock(LoadQueuePeon.class); - emitter = EasyMock.createMock(ServiceEmitter.class); - EmittingLogger.registerEmitter(emitter); - databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); - segmentsMetadataManager = EasyMock.createNiceMock(SegmentsMetadataManager.class); - - usedSegments = CreateDataSegments.ofDatasource("test") - .forIntervals(24, Granularities.HOUR) - .startingAt("2012-01-01") - .withNumPartitions(1) - .eachOfSizeInMb(1); - - ruleRunner = new RunRules(new ReplicationThrottler(24, 1, false), coordinator); - } - - @After - public void tearDown() - { - EasyMock.verify(coordinator); - EasyMock.verify(databaseRuleManager); - } - - /** - * Nodes: - * normal - 2 replicants - * maxNonPrimaryReplicantsToLoad - 10 - * Expect only 34 segments to be loaded despite there being 48 primary + non-primary replicants to load! - */ - @Test - public void testOneTierTwoReplicantsWithStrictReplicantLimit() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Collections.singletonList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("normal", 2) - ) - )).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "normal", - new ServerHolder( - new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) - .toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).withMaxNonPrimaryReplicantsToLoad(10).build()) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(34L, stats.getTieredStat("assignedCount", "normal")); - Assert.assertEquals(10L, stats.getGlobalStat("totalNonPrimaryReplicantsLoaded")); - - exec.shutdown(); - EasyMock.verify(mockPeon); - } - - /** - * Nodes: - * normal - 2 replicants - * hot - 2 replicants - * maxNonPrimaryReplicantsToLoad - 48 - * Expect only 72 segments to be loaded despite there being 96 primary + non-primary replicants to load! - */ - @Test - public void testTwoTiersTwoReplicantsWithStrictReplicantLimit() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Collections.singletonList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("hot", 2, "normal", 2) - ) - )).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "hot", - new ServerHolder( - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .addTier( - "normal", - new ServerHolder( - new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) - .toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).withMaxNonPrimaryReplicantsToLoad(48).build()) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(72L, stats.getTieredStat("assignedCount", "hot") + stats.getTieredStat("assignedCount", "normal")); - Assert.assertEquals(48L, stats.getGlobalStat("totalNonPrimaryReplicantsLoaded")); - - exec.shutdown(); - EasyMock.verify(mockPeon); - } - - /** - * Nodes: - * hot - 1 replicant - * normal - 1 replicant - * cold - 1 replicant - */ - @Test - public void testRunThreeTiersOneReplicant() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Lists.newArrayList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T06:00:00.000Z"), - ImmutableMap.of("hot", 1) - ), - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("normal", 1) - ), - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("cold", 1) - ) - )).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "hot", - new ServerHolder( - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .addTier( - "normal", - new ServerHolder( - new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .addTier( - "cold", - new ServerHolder( - new DruidServer("serverCold", "hostCold", null, 1000, ServerType.HISTORICAL, "cold", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(6L, stats.getTieredStat("assignedCount", "hot")); - Assert.assertEquals(6L, stats.getTieredStat("assignedCount", "normal")); - Assert.assertEquals(12L, stats.getTieredStat("assignedCount", "cold")); - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - exec.shutdown(); - EasyMock.verify(mockPeon); - } - - private DruidCoordinatorRuntimeParams.Builder makeCoordinatorRuntimeParams( - DruidCluster druidCluster, - BalancerStrategy balancerStrategy - ) - { - return makeCoordinatorRuntimeParams(druidCluster, balancerStrategy, usedSegments); - } - - private DruidCoordinatorRuntimeParams.Builder makeCoordinatorRuntimeParams( - DruidCluster druidCluster, - BalancerStrategy balancerStrategy, - List dataSegments - ) - { - return createCoordinatorRuntimeParams(druidCluster, dataSegments) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false)) - .withBalancerStrategy(balancerStrategy); - } - - private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams(DruidCluster druidCluster) - { - return createCoordinatorRuntimeParams(druidCluster, usedSegments); - } - - private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams(DruidCluster druidCluster, List dataSegments) - { - return CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) - .withUsedSegmentsInTest(dataSegments) - .withDatabaseRuleManager(databaseRuleManager); - } - - /** - * Nodes: - * hot - 2 replicants - * cold - 1 replicant - */ - @Test - public void testRunTwoTiersTwoReplicants() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Lists.newArrayList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T06:00:00.000Z"), - ImmutableMap.of("hot", 2) - ), - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("cold", 1) - ) - ) - ).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "hot", - new ServerHolder( - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .addTier( - "cold", - new ServerHolder( - new DruidServer("serverCold", "hostCold", null, 1000, ServerType.HISTORICAL, "cold", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(12L, stats.getTieredStat("assignedCount", "hot")); - Assert.assertEquals(18L, stats.getTieredStat("assignedCount", "cold")); - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - exec.shutdown(); - EasyMock.verify(mockPeon); - } - - /** - * Nodes: - * hot - 1 replicant - * normal - 1 replicant - */ - @Test - public void testRunTwoTiersWithExistingSegments() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Lists.newArrayList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("hot", 1) - ), - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("normal", 1) - ) - ) - ).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidServer normServer = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0); - for (DataSegment segment : usedSegments) { - normServer.addDataSegment(segment); - } - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "hot", - new ServerHolder( - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .addTier("normal", new ServerHolder(normServer.toImmutableDruidServer(), mockPeon)) - .build(); - - SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster, false); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerStrategy(balancerStrategy) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(12L, stats.getTieredStat("assignedCount", "hot")); - Assert.assertEquals(0L, stats.getTieredStat("assignedCount", "normal")); - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - exec.shutdown(); - EasyMock.verify(mockPeon); - } - - @Test - public void testRunTwoTiersTierDoesNotExist() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - emitter.emit(EasyMock.anyObject()); - EasyMock.expectLastCall().times(12); - EasyMock.replay(emitter); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Lists.newArrayList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("hot", 1) - ), - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("normal", 1) - ) - ) - ).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "normal", - new ServerHolder( - new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy) - .withEmitter(emitter) - .build(); - - ruleRunner.run(params); - - exec.shutdown(); - EasyMock.verify(emitter); - EasyMock.verify(mockPeon); - } - - @Test - public void testRunRuleDoesNotExist() - { - mockCoordinator(); - emitter.emit(EasyMock.anyObject()); - EasyMock.expectLastCall().times(1); - EasyMock.replay(emitter); - - EasyMock - .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) - .andReturn( - Collections.singletonList( - new IntervalLoadRule( - Intervals.of("2012-01-02T00:00:00.000Z/2012-01-03T00:00:00.000Z"), - ImmutableMap.of("normal", 1) - ) - ) - ) - .atLeastOnce(); - - EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).anyTimes(); - EasyMock.replay(databaseRuleManager, mockPeon); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "normal", - new ServerHolder( - new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster(), false)) - .withEmitter(emitter) - .build(); - - ruleRunner.run(params); - - EasyMock.verify(emitter, mockPeon); - } - - @Test - public void testDropRemove() - { - mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - coordinator.markSegmentAsUnused(EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - EasyMock.replay(coordinator); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Lists.newArrayList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("normal", 1) - ), - new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) - ) - ).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidServer server = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0); - for (DataSegment segment : usedSegments) { - server.addDataSegment(segment); - } - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("normal", new ServerHolder(server.toImmutableDruidServer(), mockPeon)) - .build(); - - SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster, false); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerStrategy(balancerStrategy) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(12L, stats.getGlobalStat("deletedCount")); - - exec.shutdown(); - EasyMock.verify(coordinator); - } - - @Test - public void testDropTooManyInSameTier() - { - mockCoordinator(); - mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Lists.newArrayList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("normal", 1) - ), - new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) - ) - ).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidServer server1 = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0); - server1.addDataSegment(usedSegments.get(0)); - - DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); - for (DataSegment segment : usedSegments) { - server2.addDataSegment(segment); - } - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "normal", - new ServerHolder(server1.toImmutableDruidServer(), mockPeon), - new ServerHolder(server2.toImmutableDruidServer(), mockPeon) - ) - .build(); - - SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster, false); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) - .withDynamicConfigs( - CoordinatorDynamicConfig - .builder() - .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L) - .build() - ) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerStrategy(balancerStrategy) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "normal")); - Assert.assertEquals(12L, stats.getGlobalStat("deletedCount")); - - exec.shutdown(); - EasyMock.verify(mockPeon); - } - - @Test - public void testDropTooManyInDifferentTiers() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Lists.newArrayList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("hot", 1) - ), - new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) - ) - ).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "hot", 0); - server1.addDataSegment(usedSegments.get(0)); - DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); - for (DataSegment segment : usedSegments) { - server2.addDataSegment(segment); - } - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) - .addTier("normal", new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) - .build(); - - SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster, false); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerStrategy(balancerStrategy) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "normal")); - Assert.assertEquals(12L, stats.getGlobalStat("deletedCount")); - - exec.shutdown(); - EasyMock.verify(mockPeon); - } - - @Test - public void testDontDropInDifferentTiers() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Lists.newArrayList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("hot", 1) - ), - new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) - ) - ).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "hot", 0); - DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); - for (DataSegment segment : usedSegments) { - server2.addDataSegment(segment); - } - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) - .addTier("normal", new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) - .build(); - - SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster, false); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerStrategy(balancerStrategy) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertTrue(stats.getTiers("droppedCount").isEmpty()); - Assert.assertEquals(12L, stats.getGlobalStat("deletedCount")); - - exec.shutdown(); - EasyMock.verify(mockPeon); - } - - @Test - public void testDropServerActuallyServesSegment() - { - mockCoordinator(); - EasyMock - .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) - .andReturn( - Collections.singletonList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T01:00:00.000Z"), - ImmutableMap.of("normal", 0) - ) - ) - ) - .atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "normal", 0); - server1.addDataSegment(usedSegments.get(0)); - DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); - server2.addDataSegment(usedSegments.get(1)); - DruidServer server3 = new DruidServer("serverNorm3", "hostNorm3", null, 1000, ServerType.HISTORICAL, "normal", 0); - server3.addDataSegment(usedSegments.get(1)); - server3.addDataSegment(usedSegments.get(2)); - - mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - LoadQueuePeon anotherMockPeon = EasyMock.createMock(LoadQueuePeon.class); - EasyMock.expect(anotherMockPeon.getLoadQueueSize()).andReturn(10L).atLeastOnce(); - EasyMock.expect(anotherMockPeon.getSegmentsToLoad()).andReturn(new HashSet<>()).anyTimes(); - - EasyMock.replay(anotherMockPeon); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "normal", - new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false), - new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false), - new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false) - ) - .build(); - - SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster, false); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) - .withUsedSegmentsInTest(usedSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerStrategy(balancerStrategy) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "normal")); - - exec.shutdown(); - EasyMock.verify(mockPeon); - EasyMock.verify(anotherMockPeon); - } - - /** - * Nodes: - * hot - 2 replicants - */ - @Test - public void testReplicantThrottle() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock - .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) - .andReturn( - Collections.singletonList( - new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"), - ImmutableMap.of("hot", 2) - ) - ) - ) - .atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "hot", - new ServerHolder( - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(48L, stats.getTieredStat("assignedCount", "hot")); - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - DataSegment overFlowSegment = new DataSegment( - "test", - Intervals.of("2012-02-01/2012-02-02"), - DateTimes.nowUtc().toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 1, - 0 - ); - - afterParams = ruleRunner.run( - CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) - .withEmitter(emitter) - .withUsedSegmentsInTest(overFlowSegment) - .withDatabaseRuleManager(databaseRuleManager) - .withBalancerStrategy(balancerStrategy) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster(), false)) - .build() - ); - stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(1L, stats.getTieredStat("assignedCount", "hot")); - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - EasyMock.verify(mockPeon); - exec.shutdown(); - } - - /** - * Nodes: - * hot - nothing loaded - * _default_tier - 1 segment loaded - */ - @Test - public void testReplicantThrottleAcrossTiers() - { - EasyMock - .expect(coordinator.getDynamicConfigs()) - .andReturn( - CoordinatorDynamicConfig.builder() - .withReplicationThrottleLimit(7) - .withReplicantLifetime(1) - .withMaxSegmentsInNodeLoadingQueue(1000) - .build() - ) - .atLeastOnce(); - coordinator.markSegmentAsUnused(EasyMock.anyObject()); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(coordinator); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock - .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) - .andReturn( - Collections.singletonList(new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"), - ImmutableMap.of( - "hot", 1, - DruidServer.DEFAULT_TIER, 1 - ) - ) - ) - ) - .atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "hot", - new ServerHolder( - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1) - .toImmutableDruidServer(), - mockPeon - ) - ) - .addTier( - DruidServer.DEFAULT_TIER, - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build(); - - RunRules runner = new RunRules(new ReplicationThrottler(7, 1, false), coordinator); - DruidCoordinatorRuntimeParams afterParams = runner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(24L, stats.getTieredStat("assignedCount", "hot")); - Assert.assertEquals(7L, stats.getTieredStat("assignedCount", DruidServer.DEFAULT_TIER)); - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - EasyMock.verify(mockPeon); - exec.shutdown(); - } - - @Test - public void testDropReplicantThrottle() - { - mockCoordinator(); - mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock - .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) - .andReturn( - Collections.singletonList(new IntervalLoadRule( - Intervals.of("2012-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z"), - ImmutableMap.of("normal", 1) - ) - ) - ) - .atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DataSegment overFlowSegment = new DataSegment( - "test", - Intervals.of("2012-02-01/2012-02-02"), - DateTimes.nowUtc().toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - 1, - 0 - ); - List longerUsedSegments = Lists.newArrayList(usedSegments); - longerUsedSegments.add(overFlowSegment); - - DruidServer server1 = new DruidServer("serverNorm1", "hostNorm1", null, 1000, ServerType.HISTORICAL, "normal", 0); - for (DataSegment segment : longerUsedSegments) { - server1.addDataSegment(segment); - } - DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); - for (DataSegment segment : longerUsedSegments) { - server2.addDataSegment(segment); - } - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "normal", - new ServerHolder(server1.toImmutableDruidServer(), mockPeon), - new ServerHolder(server2.toImmutableDruidServer(), mockPeon) - ) - .build(); - - SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster, false); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) - .withUsedSegmentsInTest(longerUsedSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerStrategy(balancerStrategy) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - // There is no throttling on drop - Assert.assertEquals(25L, stats.getTieredStat("droppedCount", "normal")); - EasyMock.verify(mockPeon); - exec.shutdown(); - } - - @Test - public void testRulesRunOnNonOvershadowedSegmentsOnly() - { - Set usedSegments = new HashSet<>(); - DataSegment v1 = new DataSegment( - "test", - Intervals.of("2012-01-01/2012-01-02"), - "1", - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 1 - ); - DataSegment v2 = new DataSegment( - "test", - Intervals.of("2012-01-01/2012-01-02"), - "2", - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 1 - ); - usedSegments.add(v1); - usedSegments.add(v2); - - mockCoordinator(); - mockPeon.loadSegment(EasyMock.eq(v2), EasyMock.anyObject()); - EasyMock.expectLastCall().once(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Collections.singletonList(new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1)))).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - DruidServer.DEFAULT_TIER, - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) - .withUsedSegmentsInTest(usedSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster(), false)) - .withBalancerStrategy(balancerStrategy) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(1, stats.getTiers("assignedCount").size()); - Assert.assertEquals(1, stats.getTieredStat("assignedCount", "_default_tier")); - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - Assert.assertEquals(2, usedSegments.size()); - Assert.assertEquals(usedSegments, params.getUsedSegments()); - Assert.assertEquals(usedSegments, afterParams.getUsedSegments()); - - EasyMock.verify(mockPeon); - exec.shutdown(); - } - - /** - * Tier - __default_tier - * Nodes - 2 - * Replicants - 3 - * Random balancer strategy should not assign anything and not get into loop as there are not enough nodes for replication - */ - @Test(timeout = 5000L) - public void testTwoNodesOneTierThreeReplicantsRandomStrategyNotEnoughNodes() - { - mockCoordinator(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Collections.singletonList( - new ForeverLoadRule( - ImmutableMap.of(DruidServer.DEFAULT_TIER, 3) - ) - )).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DataSegment dataSegment = new DataSegment( - "test", - Intervals.utc(0, 1), - DateTimes.nowUtc().toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 1 - ); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - DruidServer.DEFAULT_TIER, - new ServerHolder( - new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0).addDataSegment(dataSegment) - .toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer("server2", "host2", null, 1000, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0).addDataSegment(dataSegment) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - RandomBalancerStrategy balancerStrategy = new RandomBalancerStrategy(); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy, Collections.singletonList(dataSegment)) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - - Assert.assertEquals(0L, stats.getTieredStat("assignedCount", DruidServer.DEFAULT_TIER)); - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - EasyMock.verify(mockPeon); - } - - - /** - * Tier - __default_tier - * Nodes - 1 - * Replicants - 1 - * Random balancer strategy should select the only node - */ - @Test(timeout = 5000L) - public void testOneNodesOneTierOneReplicantRandomStrategyEnoughSpace() - { - mockCoordinator(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - mockEmptyPeon(); - - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Collections.singletonList( - new ForeverLoadRule( - ImmutableMap.of(DruidServer.DEFAULT_TIER, 1) - ) - )).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DataSegment dataSegment = new DataSegment( - "test", - Intervals.utc(0, 1), - DateTimes.nowUtc().toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 1 - ); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - DruidServer.DEFAULT_TIER, - new ServerHolder( - new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - RandomBalancerStrategy balancerStrategy = new RandomBalancerStrategy(); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy, Collections.singletonList(dataSegment)) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertEquals(1L, stats.getTieredStat("assignedCount", DruidServer.DEFAULT_TIER)); - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - EasyMock.verify(mockPeon); - } - - /** - * Tier - __default_tier - * Nodes - 1 - * Replicants - 1 - * Random balancer strategy should not assign anything as there is not enough space - */ - @Test(timeout = 5000L) - public void testOneNodesOneTierOneReplicantRandomStrategyNotEnoughSpace() - { - mockCoordinator(); - mockEmptyPeon(); - int numReplicants = 1; - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Collections.singletonList( - new ForeverLoadRule( - ImmutableMap.of(DruidServer.DEFAULT_TIER, numReplicants) - ) - )).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DataSegment dataSegment = new DataSegment( - "test", - Intervals.utc(0, 1), - DateTimes.nowUtc().toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 11 - ); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - DruidServer.DEFAULT_TIER, - new ServerHolder( - new DruidServer("server1", "host1", null, 10, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - RandomBalancerStrategy balancerStrategy = new RandomBalancerStrategy(); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy, Collections.singletonList(dataSegment)) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertEquals(dataSegment.getSize() * numReplicants, stats.getTieredStat(LoadRule.REQUIRED_CAPACITY, DruidServer.DEFAULT_TIER)); - Assert.assertTrue(stats.getTiers("assignedCount").isEmpty()); // since primary assignment failed - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - EasyMock.verify(mockPeon); - } - - /** - * Tier - __default_tier - * Nodes - 1 - * Replicants - 1 - * Cost balancer strategy should not assign anything as there is not enough space - */ - @Test - public void testOneNodesOneTierOneReplicantCostBalancerStrategyNotEnoughSpace() - { - mockCoordinator(); - mockEmptyPeon(); - int numReplicants = 1; - EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Collections.singletonList( - new ForeverLoadRule( - ImmutableMap.of(DruidServer.DEFAULT_TIER, numReplicants) - ) - )).atLeastOnce(); - EasyMock.replay(databaseRuleManager); - - DataSegment dataSegment = new DataSegment( - "test", - Intervals.utc(0, 1), - DateTimes.nowUtc().toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 11 - ); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - DruidServer.DEFAULT_TIER, - new ServerHolder( - new DruidServer("server1", "host1", null, 10, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0) - .toImmutableDruidServer(), - mockPeon - ) - ) - .build(); - - ListeningExecutorService exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); - CostBalancerStrategy balancerStrategy = new CostBalancerStrategy(exec); - - DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy, Collections.singletonList(dataSegment)) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .build(); - - DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); - CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertEquals(dataSegment.getSize() * numReplicants, stats.getTieredStat(LoadRule.REQUIRED_CAPACITY, DruidServer.DEFAULT_TIER)); - Assert.assertTrue(stats.getTiers("assignedCount").isEmpty()); // since primary assignment should fail - Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); - Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - - exec.shutdown(); - EasyMock.verify(mockPeon); - } - - private void mockCoordinator() - { - EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - coordinator.markSegmentAsUnused(EasyMock.anyObject()); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(coordinator, segmentsMetadataManager); - } - - private void mockEmptyPeon() - { - EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(new HashSet<>()).anyTimes(); - EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(new HashSet<>()).anyTimes(); - EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce(); - EasyMock.expect(mockPeon.getNumberOfSegmentsInQueue()).andReturn(0).anyTimes(); - EasyMock.replay(mockPeon); - } - - private CoordinatorDynamicConfig createCoordinatorDynamicConfig() - { - return CoordinatorDynamicConfig.builder() - .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0) - .withMergeBytesLimit(0) - .withMergeSegmentsLimit(0) - .withMaxSegmentsToMove(0) - .withReplicantLifetime(1) - .withReplicationThrottleLimit(24) - .withBalancerComputeThreads(0) - .withEmitBalancingStats(false) - .withSpecificDataSourcesToKillUnusedSegmentsIn(null) - .withMaxSegmentsInNodeLoadingQueue(1000) - .withPauseCoordination(false) - .build(); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ServerHolderTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ServerHolderTest.java index cb1ee0425b6..ed9735c6bcc 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ServerHolderTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ServerHolderTest.java @@ -26,6 +26,7 @@ import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.junit.Assert; @@ -115,8 +116,8 @@ public class ServerHolderTest ); Assert.assertEquals(0, h1.compareTo(h2)); - Assert.assertEquals(-1, h3.compareTo(h1)); - Assert.assertEquals(-1, h3.compareTo(h4)); + Assert.assertEquals(1, h3.compareTo(h1)); + Assert.assertEquals(1, h3.compareTo(h4)); } @Test @@ -192,7 +193,7 @@ public class ServerHolderTest ); Assert.assertTrue(h1.isServingSegment(SEGMENTS.get(0))); Assert.assertFalse(h1.isServingSegment(SEGMENTS.get(1))); - Assert.assertTrue(h1.isServingSegment(SEGMENTS.get(0).getId())); - Assert.assertFalse(h1.isServingSegment(SEGMENTS.get(1).getId())); + Assert.assertTrue(h1.hasSegmentLoaded(SEGMENTS.get(0).getId())); + Assert.assertFalse(h1.hasSegmentLoaded(SEGMENTS.get(1).getId())); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/BalancerStrategyTest.java similarity index 80% rename from server/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyTest.java rename to server/src/test/java/org/apache/druid/server/coordinator/balancer/BalancerStrategyTest.java index b4d3ac5d11d..ad4d7f65aed 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/BalancerStrategyTest.java @@ -17,12 +17,14 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.junit.Assert; @@ -33,6 +35,7 @@ import org.junit.runners.Parameterized; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -82,31 +85,33 @@ public class BalancerStrategyTest final ServerHolder serverHolder = new ServerHolder( new DruidServer("server1", "host1", null, 10L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0).addDataSegment(proposedDataSegment).toImmutableDruidServer(), new LoadQueuePeonTester()); - serverHolders = new ArrayList<>(); - serverHolders.add(serverHolder); - final ServerHolder foundServerHolder = balancerStrategy.findNewSegmentHomeReplicator(proposedDataSegment, serverHolders); - // since there is not enough space on server having available size 10L to host a segment of size 11L, it should be null - Assert.assertNull(foundServerHolder); + Assert.assertFalse( + balancerStrategy.findServersToLoadSegment( + proposedDataSegment, + Collections.singletonList(serverHolder) + ).hasNext() + ); } @Test(timeout = 5000L) public void findNewSegmentHomeReplicatorNotEnoughNodesForReplication() { final ServerHolder serverHolder1 = new ServerHolder( - new DruidServer("server1", "host1", null, 1000L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0).addDataSegment(proposedDataSegment).toImmutableDruidServer(), + new DruidServer("server1", "host1", null, 1000L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0) + .addDataSegment(proposedDataSegment).toImmutableDruidServer(), new LoadQueuePeonTester()); final ServerHolder serverHolder2 = new ServerHolder( - new DruidServer("server2", "host2", null, 1000L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0).addDataSegment(proposedDataSegment).toImmutableDruidServer(), + new DruidServer("server2", "host2", null, 1000L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0) + .addDataSegment(proposedDataSegment).toImmutableDruidServer(), new LoadQueuePeonTester()); serverHolders = new ArrayList<>(); serverHolders.add(serverHolder1); serverHolders.add(serverHolder2); - final ServerHolder foundServerHolder = balancerStrategy.findNewSegmentHomeReplicator(proposedDataSegment, serverHolders); // since there is not enough nodes to load 3 replicas of segment - Assert.assertNull(foundServerHolder); + Assert.assertFalse(balancerStrategy.findServersToLoadSegment(proposedDataSegment, serverHolders).hasNext()); } @Test @@ -117,7 +122,8 @@ public class BalancerStrategyTest new LoadQueuePeonTester()); serverHolders = new ArrayList<>(); serverHolders.add(serverHolder); - final ServerHolder foundServerHolder = balancerStrategy.findNewSegmentHomeReplicator(proposedDataSegment, serverHolders); + final ServerHolder foundServerHolder = balancerStrategy + .findServersToLoadSegment(proposedDataSegment, serverHolders).next(); // since there is enough space on server it should be selected Assert.assertEquals(serverHolder, foundServerHolder); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyTest.java similarity index 92% rename from server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java rename to server/src/test/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyTest.java index 13942b010eb..f4cad1cb5cd 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.cost; +package org.apache.druid.server.coordinator.balancer; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; @@ -25,10 +25,8 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CachingCostBalancerStrategy; -import org.apache.druid.server.coordinator.CostBalancerStrategy; -import org.apache.druid.server.coordinator.LoadQueuePeonTester; import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -95,12 +93,15 @@ public class CachingCostBalancerStrategyTest executorService ); CostBalancerStrategy costBalancerStrategy = createCostBalancerStrategy(executorService); + final ServerHolder firstServer = serverHolderList.get(0); int notEqual = segmentQueries .stream() .mapToInt( s -> { - ServerHolder s1 = cachingCostBalancerStrategy.findNewSegmentHomeBalancer(s, serverHolderList); - ServerHolder s2 = costBalancerStrategy.findNewSegmentHomeBalancer(s, serverHolderList); + ServerHolder s1 = cachingCostBalancerStrategy + .findDestinationServerToMoveSegment(s, firstServer, serverHolderList); + ServerHolder s2 = costBalancerStrategy + .findDestinationServerToMoveSegment(s, firstServer, serverHolderList); return (s1.getServer().getName().equals(s2.getServer().getName())) ? 0 : 1; } ) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java similarity index 94% rename from server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java rename to server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java index 57735477486..6580425764a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.DruidServer; @@ -28,6 +28,9 @@ import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.timeline.DataSegment; import org.junit.After; @@ -320,13 +323,14 @@ public class CostBalancerStrategyTest .startingAt("2012-10-24") .eachOfSizeInMb(100).get(0); - ServerHolder serverA = new ServerHolder(createHistorical().toImmutableDruidServer(), null); - ServerHolder serverB = new ServerHolder(createHistorical().toImmutableDruidServer(), null); + final LoadQueuePeonTester peon = new LoadQueuePeonTester(); + ServerHolder serverA = new ServerHolder(createHistorical().toImmutableDruidServer(), peon); + ServerHolder serverB = new ServerHolder(createHistorical().toImmutableDruidServer(), peon); balancerExecutor.shutdownNow(); Assert.assertThrows( RejectedExecutionException.class, - () -> strategy.findNewSegmentHomeBalancer(segment, Arrays.asList(serverA, serverB)) + () -> strategy.findServersToLoadSegment(segment, Arrays.asList(serverA, serverB)) ); } @@ -338,14 +342,22 @@ public class CostBalancerStrategyTest .startingAt("2012-10-24") .eachOfSizeInMb(100).get(0); - ServerHolder serverA = new ServerHolder(createHistorical().toImmutableDruidServer(), null); - ServerHolder serverB = new ServerHolder(createHistorical().toImmutableDruidServer(), null); + final LoadQueuePeonTester peon = new LoadQueuePeonTester(); + ServerHolder serverA = new ServerHolder(createHistorical().toImmutableDruidServer(), peon); + ServerHolder serverB = new ServerHolder(createHistorical().toImmutableDruidServer(), peon); - strategy.findNewSegmentHomeBalancer(segment, Arrays.asList(serverA, serverB)); + strategy.findServersToLoadSegment(segment, Arrays.asList(serverA, serverB)); List events = serviceEmitter.getEvents(); Assert.assertEquals(1, events.size()); Assert.assertTrue(events.get(0) instanceof AlertEvent); + + AlertEvent alertEvent = (AlertEvent) events.get(0); + Assert.assertEquals( + "Cost balancer strategy timed out in action [findServersToLoadSegment]." + + " Try setting a higher value of 'balancerComputeThreads'.", + alertEvent.getDescription() + ); } private void verifyServerCosts( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategyTest.java similarity index 94% rename from server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java rename to server/src/test/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategyTest.java index 9a73e58e12e..98f86ec3181 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategyTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.MoreExecutors; @@ -28,6 +28,8 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -134,7 +136,7 @@ public class DiskNormalizedCostBalancerStrategyTest BalancerStrategy strategy = new DiskNormalizedCostBalancerStrategy( MoreExecutors.listeningDecorator(Execs.multiThreaded(4, "DiskNormalizedCostBalancerStrategyTest-%d")) ); - ServerHolder holder = strategy.findNewSegmentHomeReplicator(segment, serverHolderList); + ServerHolder holder = strategy.findServersToLoadSegment(segment, serverHolderList).next(); Assert.assertNotNull("Should be able to find a place for new segment!!", holder); Assert.assertEquals("Best Server should be BEST_SERVER", "BEST_SERVER", holder.getServer().getName()); } @@ -148,7 +150,7 @@ public class DiskNormalizedCostBalancerStrategyTest BalancerStrategy strategy = new DiskNormalizedCostBalancerStrategy( MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "DiskNormalizedCostBalancerStrategyTest-%d")) ); - ServerHolder holder = strategy.findNewSegmentHomeReplicator(segment, serverHolderList); + ServerHolder holder = strategy.findServersToLoadSegment(segment, serverHolderList).next(); Assert.assertNotNull("Should be able to find a place for new segment!!", holder); Assert.assertEquals("Best Server should be BEST_SERVER", "BEST_SERVER", holder.getServer().getName()); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSamplerTest.java similarity index 75% rename from server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java rename to server/src/test/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSamplerTest.java index 4181ba2bbe7..25b18015468 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSamplerTest.java @@ -17,23 +17,30 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.balancer; import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; +import org.apache.druid.server.coordinator.loading.SegmentAction; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; public class ReservoirSegmentSamplerTest { @@ -48,6 +55,9 @@ public class ReservoirSegmentSamplerTest .withNumPartitions(10) .eachOfSizeInMb(100); + private final Function> GET_SERVED_SEGMENTS + = serverHolder -> serverHolder.getServer().iterateAllSegments(); + @Before public void setUp() { @@ -70,7 +80,7 @@ public class ReservoirSegmentSamplerTest // due to the pseudo-randomness of this method, we may not select a segment every single time no matter what. segmentCountMap.compute( ReservoirSegmentSampler - .getRandomBalancerSegmentHolders(servers, Collections.emptySet(), 1) + .pickMovableSegmentsFrom(servers, 1, GET_SERVED_SEGMENTS, Collections.emptySet()) .get(0).getSegment(), (segment, count) -> count == null ? 1 : count + 1 ); @@ -114,6 +124,42 @@ public class ReservoirSegmentSamplerTest Assert.assertEquals(3, segmentCountMap.size()); } + @Test + public void testPickLoadingOrLoadedSegments() + { + final List loadedSegments = Arrays.asList(segments.get(0), segments.get(1)); + final List loadingSegments = Arrays.asList(segments.get(2), segments.get(3)); + + final ServerHolder server1 = createHistorical("server1", loadedSegments.get(0)); + server1.startOperation(SegmentAction.LOAD, loadingSegments.get(0)); + + final ServerHolder server2 = createHistorical("server2", loadedSegments.get(1)); + server2.startOperation(SegmentAction.LOAD, loadingSegments.get(1)); + + // Pick only loading segments + Set pickedSegments = ReservoirSegmentSampler + .pickMovableSegmentsFrom( + Arrays.asList(server1, server2), + 10, + ServerHolder::getLoadingSegments, + Collections.emptySet() + ) + .stream().map(BalancerSegmentHolder::getSegment).collect(Collectors.toSet()); + + // Verify that only loading segments are picked + Assert.assertEquals(loadingSegments.size(), pickedSegments.size()); + Assert.assertTrue(pickedSegments.containsAll(loadingSegments)); + + // Pick only loaded segments + pickedSegments = ReservoirSegmentSampler + .pickMovableSegmentsFrom(Arrays.asList(server1, server2), 10, GET_SERVED_SEGMENTS, Collections.emptySet()) + .stream().map(BalancerSegmentHolder::getSegment).collect(Collectors.toSet()); + + // Verify that only loaded segments are picked + Assert.assertEquals(loadedSegments.size(), pickedSegments.size()); + Assert.assertTrue(pickedSegments.containsAll(loadedSegments)); + } + @Test public void testSegmentsOnBrokersAreIgnored() { @@ -128,16 +174,17 @@ public class ReservoirSegmentSamplerTest ); // Try to pick all the segments on the servers - List pickedSegments = ReservoirSegmentSampler.getRandomBalancerSegmentHolders( + List pickedSegments = ReservoirSegmentSampler.pickMovableSegmentsFrom( Arrays.asList(historical, broker), - Collections.emptySet(), - 10 + 10, + GET_SERVED_SEGMENTS, + Collections.emptySet() ); // Verify that only the segments on the historical are picked Assert.assertEquals(2, pickedSegments.size()); for (BalancerSegmentHolder holder : pickedSegments) { - Assert.assertEquals(historical.getServer(), holder.getFromServer()); + Assert.assertEquals(historical, holder.getServer()); } } @@ -160,7 +207,7 @@ public class ReservoirSegmentSamplerTest // Try to pick all the segments on the servers List pickedSegments = ReservoirSegmentSampler - .getRandomBalancerSegmentHolders(servers, Collections.singleton(broadcastDatasource), 10); + .pickMovableSegmentsFrom(servers, 10, GET_SERVED_SEGMENTS, Collections.singleton(broadcastDatasource)); // Verify that none of the broadcast segments are picked Assert.assertEquals(2, pickedSegments.size()); @@ -212,7 +259,7 @@ public class ReservoirSegmentSamplerTest int numIterations = 1; for (; numIterations < 10000; ++numIterations) { ReservoirSegmentSampler - .getRandomBalancerSegmentHolders(servers, Collections.emptySet(), sampleSize) + .pickMovableSegmentsFrom(servers, sampleSize, GET_SERVED_SEGMENTS, Collections.emptySet()) .forEach(holder -> pickedSegments.add(holder.getSegment())); if (pickedSegments.size() >= numSegments) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/cost/SegmentsCostCacheTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/SegmentsCostCacheTest.java similarity index 99% rename from server/src/test/java/org/apache/druid/server/coordinator/cost/SegmentsCostCacheTest.java rename to server/src/test/java/org/apache/druid/server/coordinator/balancer/SegmentsCostCacheTest.java index f0ae22094fe..3db763bea25 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/cost/SegmentsCostCacheTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/SegmentsCostCacheTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.cost; +package org.apache.druid.server.coordinator.balancer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.timeline.DataSegment; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java new file mode 100644 index 00000000000..a506443e8ed --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java @@ -0,0 +1,491 @@ +/* + * 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.druid.server.coordinator.duty; + +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.client.DruidServer; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.balancer.BalancerStrategy; +import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.Set; + +public class BalanceSegmentsTest +{ + private SegmentLoadQueueManager loadQueueManager; + + private DataSegment segment1; + private DataSegment segment2; + private DataSegment segment3; + private DataSegment segment4; + private DataSegment segment5; + + private DataSegment[] allSegments; + + private ListeningExecutorService balancerStrategyExecutor; + private BalancerStrategy balancerStrategy; + private Set broadcastDatasources; + + private DruidServer server1; + private DruidServer server2; + private DruidServer server3; + private DruidServer server4; + + @Before + public void setUp() + { + loadQueueManager = new SegmentLoadQueueManager(null, null, null); + + // Create test segments for multiple datasources + final DateTime start1 = DateTimes.of("2012-01-01"); + final DateTime start2 = DateTimes.of("2012-02-01"); + final String version = DateTimes.of("2012-03-01").toString(); + + segment1 = createHourlySegment("datasource1", start1, version); + segment2 = createHourlySegment("datasource1", start2, version); + segment3 = createHourlySegment("datasource2", start1, version); + segment4 = createHourlySegment("datasource2", start2, version); + segment5 = createHourlySegment("datasourceBroadcast", start2, version); + allSegments = new DataSegment[]{segment1, segment2, segment3, segment4, segment5}; + + server1 = new DruidServer("server1", "server1", null, 100L, ServerType.HISTORICAL, "normal", 0); + server2 = new DruidServer("server2", "server2", null, 100L, ServerType.HISTORICAL, "normal", 0); + server3 = new DruidServer("server3", "server3", null, 100L, ServerType.HISTORICAL, "normal", 0); + server4 = new DruidServer("server4", "server4", null, 100L, ServerType.HISTORICAL, "normal", 0); + + balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d")); + balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor); + + broadcastDatasources = Collections.singleton("datasourceBroadcast"); + } + + @After + public void tearDown() + { + balancerStrategyExecutor.shutdownNow(); + } + + @Test + public void testMoveToEmptyServerBalancer() + { + final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2, segment3, segment4); + final ServerHolder serverHolder2 = createHolder(server2); + + DruidCoordinatorRuntimeParams params = + defaultRuntimeParamsBuilder(serverHolder1, serverHolder2) + .withBalancerStrategy(balancerStrategy) + .withBroadcastDatasources(broadcastDatasources) + .build(); + + CoordinatorRunStats stats = runBalancer(params); + long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") + + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"); + Assert.assertEquals(2L, totalMoved); + } + + /** + * Server 1 has 2 segments. + * Server 2 (decommissioning) has 2 segments. + * Server 3 is empty. + * Decommissioning percent is 60. + * Max segments to move is 3. + * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1. + */ + @Test + public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove() + { + final ServerHolder serverHolder1 = createHolder(server1, false, segment1, segment2); + final ServerHolder serverHolder2 = createHolder(server2, true, segment3, segment4); + final ServerHolder serverHolder3 = createHolder(server3, false); + + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + expectFindDestinationAndReturn(strategy, serverHolder3); + EasyMock.replay(strategy); + + // ceil(3 * 0.6) = 2 segments from decommissioning servers + CoordinatorDynamicConfig dynamicConfig = + CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withMaxSegmentsToMove(3) + .withDecommissioningMaxPercentOfMaxSegmentsToMove(60) + .build(); + DruidCoordinatorRuntimeParams params = + defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3) + .withDynamicConfigs(dynamicConfig) + .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runBalancer(params); + + EasyMock.verify(strategy); + + // 2 segments are moved from the decommissioning server and 1 from the active server + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")); + Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2")); + Set segmentsMoved = serverHolder3.getPeon().getSegmentsToLoad(); + Assert.assertTrue(segmentsMoved.contains(segment3)); + Assert.assertTrue(segmentsMoved.contains(segment4)); + Assert.assertTrue(segmentsMoved.contains(segment1) || segmentsMoved.contains(segment2)); + } + + @Test + public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove() + { + final ServerHolder holder1 = createHolder(server1, false, segment1, segment2); + final ServerHolder holder2 = createHolder(server2, true, segment3, segment4); + final ServerHolder holder3 = createHolder(server3, false); + + CoordinatorDynamicConfig dynamicConfig = + CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withDecommissioningMaxPercentOfMaxSegmentsToMove(0) + .withMaxSegmentsToMove(1).build(); + DruidCoordinatorRuntimeParams params = + defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build(); + + CoordinatorRunStats stats = runBalancer(params); + + // Verify that either segment1 or segment2 is chosen for move + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource())); + DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next(); + Assert.assertEquals(segment1.getDataSource(), movingSegment.getDataSource()); + } + + @Test + public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove() + { + final ServerHolder holder1 = createHolder(server1, false, segment1, segment2); + final ServerHolder holder2 = createHolder(server2, true, segment3, segment4); + final ServerHolder holder3 = createHolder(server3, false); + + CoordinatorDynamicConfig dynamicConfig = + CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withDecommissioningMaxPercentOfMaxSegmentsToMove(100) + .withMaxSegmentsToMove(1).build(); + DruidCoordinatorRuntimeParams params = + defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build(); + + CoordinatorRunStats stats = runBalancer(params); + + // Verify that either segment3 or segment4 is chosen for move + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment3.getDataSource())); + DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next(); + Assert.assertEquals(segment3.getDataSource(), movingSegment.getDataSource()); + } + + /** + * Should balance segments as usual (ignoring percent) with empty decommissioningNodes. + */ + @Test + public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning() + { + final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2); + final ServerHolder serverHolder2 = createHolder(server2, segment3, segment4); + final ServerHolder serverHolder3 = createHolder(server3); + + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + expectFindDestinationAndReturn(strategy, serverHolder3); + EasyMock.replay(strategy); + + CoordinatorDynamicConfig dynamicConfig = + CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withMaxSegmentsToMove(3) + .withDecommissioningMaxPercentOfMaxSegmentsToMove(9) + .build(); + DruidCoordinatorRuntimeParams params = + defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3) + .withDynamicConfigs(dynamicConfig) + .withBalancerStrategy(strategy) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runBalancer(params); + EasyMock.verify(strategy); + long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") + + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"); + Assert.assertEquals(3L, totalMoved); + Assert.assertEquals(3, serverHolder3.getPeon().getSegmentsToLoad().size()); + } + + /** + * Shouldn't move segments to a decommissioning server. + */ + @Test + public void testMoveToDecommissioningServer() + { + final ServerHolder activeServer = createHolder(server1, false, allSegments); + final ServerHolder decommissioningServer = createHolder(server2, true); + + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + expectFindDestinationAndReturn(strategy, decommissioningServer); + EasyMock.replay(strategy); + + DruidCoordinatorRuntimeParams params = + defaultRuntimeParamsBuilder(activeServer, decommissioningServer) + .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) + .build(); + + CoordinatorRunStats stats = runBalancer(params); + EasyMock.verify(strategy); + Assert.assertFalse(stats.hasStat(Stats.Segments.MOVED)); + } + + @Test + public void testMoveFromDecommissioningServer() + { + final ServerHolder decommissioningServer = createHolder(server1, true, allSegments); + final ServerHolder activeServer = createHolder(server2); + + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + expectFindDestinationAndReturn(strategy, activeServer); + EasyMock.replay(strategy); + + DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(decommissioningServer, activeServer) + .withDynamicConfigs( + CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withMaxSegmentsToMove(3).build() + ) + .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) + .build(); + + runBalancer(params); + EasyMock.verify(strategy); + Assert.assertEquals(0, decommissioningServer.getPeon().getSegmentsToLoad().size()); + Assert.assertEquals(3, activeServer.getPeon().getSegmentsToLoad().size()); + } + + @Test + public void testMoveMaxLoadQueueServerBalancer() + { + final int maxSegmentsInQueue = 1; + final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, false, allSegments); + final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue, false); + + final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig + .builder() + .withSmartSegmentLoading(false) + .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue) + .build(); + DruidCoordinatorRuntimeParams params = + defaultRuntimeParamsBuilder(holder1, holder2) + .withDynamicConfigs(dynamicConfig) + .build(); + + CoordinatorRunStats stats = runBalancer(params); + + // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1 + Assert.assertEquals(maxSegmentsInQueue, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")); + } + + @Test + public void testRun1() + { + // Mock some servers of different usages + DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( + createHolder(server1, allSegments), + createHolder(server2) + ).build(); + + CoordinatorRunStats stats = runBalancer(params); + Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0); + } + + @Test + public void testRun2() + { + // Mock some servers of different usages + DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( + createHolder(server1, allSegments), + createHolder(server2), + createHolder(server3), + createHolder(server4) + ).build(); + + CoordinatorRunStats stats = runBalancer(params); + Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0); + } + + @Test + public void testMaxSegmentsToMoveIsHonored() + { + // Move from non-decomissioning servers + final ServerHolder holder1 = createHolder(server1, segment1, segment2); + final ServerHolder holder2 = createHolder(server2, segment3, segment4); + final ServerHolder holder3 = createHolder(server3); + + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + expectFindDestinationAndReturn(strategy, holder3); + EasyMock.replay(strategy); + + DruidCoordinatorRuntimeParams params = + defaultRuntimeParamsBuilder(holder1, holder2, holder3) + .withDynamicConfigs( + CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withMaxSegmentsToMove(1) + .withUseBatchedSegmentSampler(true) + .withPercentOfSegmentsToConsiderPerMove(40) + .build() + ) + .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) + .build(); + + CoordinatorRunStats stats = runBalancer(params); + EasyMock.verify(strategy); + long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") + + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"); + Assert.assertEquals(1L, totalMoved); + Assert.assertEquals(1, holder3.getPeon().getSegmentsToLoad().size()); + } + + @Test + public void testUseBatchedSegmentSampler() + { + DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( + createHolder(server1, allSegments), + createHolder(server2), + createHolder(server3), + createHolder(server4) + ) + .withDynamicConfigs( + CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withMaxSegmentsToMove(2) + .withUseBatchedSegmentSampler(true) + .build() + ) + .withBroadcastDatasources(broadcastDatasources) + .build(); + + CoordinatorRunStats stats = runBalancer(params); + long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") + + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"); + Assert.assertEquals(2L, totalMoved); + } + + private CoordinatorRunStats runBalancer(DruidCoordinatorRuntimeParams params) + { + params = new BalanceSegments().run(params); + if (params == null) { + Assert.fail("BalanceSegments duty returned null params"); + return new CoordinatorRunStats(); + } else { + return params.getCoordinatorStats(); + } + } + + private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( + ServerHolder... servers + ) + { + return DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc()) + .withDruidCluster(DruidCluster.builder().addTier("normal", servers).build()) + .withUsedSegmentsInTest(allSegments) + .withBroadcastDatasources(broadcastDatasources) + .withBalancerStrategy(balancerStrategy) + .withSegmentAssignerUsing(loadQueueManager); + } + + private ServerHolder createHolder(DruidServer server, DataSegment... loadedSegments) + { + return createHolder(server, false, loadedSegments); + } + + private ServerHolder createHolder(DruidServer server, boolean isDecommissioning, DataSegment... loadedSegments) + { + return createHolder(server, 0, isDecommissioning, loadedSegments); + } + + private ServerHolder createHolder( + DruidServer server, + int maxSegmentsInLoadQueue, + boolean isDecommissioning, + DataSegment... loadedSegments + ) + { + for (DataSegment segment : loadedSegments) { + server.addDataSegment(segment); + } + + return new ServerHolder( + server.toImmutableDruidServer(), + new LoadQueuePeonTester(), + isDecommissioning, + maxSegmentsInLoadQueue, + 10 + ); + } + + private void expectFindDestinationAndReturn(BalancerStrategy strategy, ServerHolder chosenServer) + { + EasyMock.expect( + strategy.findDestinationServerToMoveSegment( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + ) + ).andReturn(chosenServer).anyTimes(); + } + + private DataSegment createHourlySegment(String datasource, DateTime start, String version) + { + return new DataSegment( + datasource, + new Interval(start, start.plusHours(1)), + version, + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + NoneShardSpec.instance(), + 0, + 8L + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java new file mode 100644 index 00000000000..1090ef573e4 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStatsTest.java @@ -0,0 +1,71 @@ +/* + * 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.druid.server.coordinator.duty; + +import it.unimi.dsi.fastutil.objects.Object2IntMaps; +import it.unimi.dsi.fastutil.objects.Object2LongMaps; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.balancer.RandomBalancerStrategy; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Stats; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Collections; + +@RunWith(MockitoJUnitRunner.class) +public class CollectSegmentAndServerStatsTest +{ + @Mock + private DruidCoordinator mockDruidCoordinator; + + @Test + public void testCollectedSegmentStats() + { + DruidCoordinatorRuntimeParams runtimeParams = + DruidCoordinatorRuntimeParams.newBuilder(DateTimes.nowUtc()) + .withDruidCluster(DruidCluster.EMPTY) + .withUsedSegmentsInTest() + .withBalancerStrategy(new RandomBalancerStrategy()) + .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null, null)) + .build(); + + Mockito.when(mockDruidCoordinator.getDatasourceToUnavailableSegmentCount()) + .thenReturn(Object2IntMaps.singleton("ds", 10)); + Mockito.when(mockDruidCoordinator.getTierToDatasourceToUnderReplicatedCount(false)) + .thenReturn(Collections.singletonMap("ds", Object2LongMaps.singleton("tier1", 100))); + + CoordinatorDuty duty = new CollectSegmentAndServerStats(mockDruidCoordinator); + DruidCoordinatorRuntimeParams params = duty.run(runtimeParams); + + CoordinatorRunStats stats = params.getCoordinatorStats(); + Assert.assertTrue(stats.hasStat(Stats.Segments.UNAVAILABLE)); + Assert.assertTrue(stats.hasStat(Stats.Segments.UNDER_REPLICATED)); + } + +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 7e87ea62dea..14fcd05d005 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -74,8 +74,6 @@ import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; -import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; -import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -84,6 +82,8 @@ import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.apache.druid.server.coordinator.UserCompactionTaskTransformConfig; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; @@ -122,6 +122,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.function.BooleanSupplier; import java.util.function.Supplier; @@ -178,7 +179,7 @@ public class CompactSegmentsTest private final PartitionsSpec partitionsSpec; private final BiFunction shardSpecFactory; - private Map dataSources; + private DataSourcesSnapshot dataSources; Map> datasourceToSegments = new HashMap<>(); public CompactSegmentsTest(PartitionsSpec partitionsSpec, BiFunction shardSpecFactory) @@ -205,9 +206,7 @@ public class CompactSegmentsTest } } } - dataSources = DataSourcesSnapshot - .fromUsedSegments(allSegments, ImmutableMap.of()) - .getUsedSegmentsTimelinesPerDataSource(); + dataSources = DataSourcesSnapshot.fromUsedSegments(allSegments, ImmutableMap.of()); Mockito.when(COORDINATOR_CONFIG.getCompactionSkipLockedIntervals()).thenReturn(true); } @@ -234,8 +233,8 @@ public class CompactSegmentsTest interval, "version", null, - ImmutableList.of(), - ImmutableList.of(), + Collections.emptyList(), + Collections.emptyList(), shardSpec, 0, 10L @@ -260,8 +259,6 @@ public class CompactSegmentsTest CompactSegments serdeCompactSegments = JSON_MAPPER.readValue(compactSegmentString, CompactSegments.class); Assert.assertNotNull(serdeCompactSegments); - Assert.assertEquals(COORDINATOR_CONFIG.getCompactionSkipLockedIntervals(), serdeCompactSegments.isSkipLockedIntervals()); - Assert.assertEquals(indexingServiceClient, serdeCompactSegments.getIndexingServiceClient()); } @Test @@ -358,10 +355,10 @@ public class CompactSegmentsTest doCompactionAndAssertCompactSegmentStatistics(compactSegments, compactionRunCount); } // Test that stats does not change (and is still correct) when auto compaction runs with everything is fully compacted - final CoordinatorStats stats = doCompactSegments(compactSegments); + final CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( 0, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); for (int i = 0; i < 3; i++) { verifySnapshot( @@ -407,7 +404,7 @@ public class CompactSegmentsTest doCompactSegments(compactSegments, new ArrayList<>()); Assert.assertEquals( 0, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); Assert.assertTrue(compactSegments.getAutoCompactionSnapshot().isEmpty()); @@ -439,10 +436,7 @@ public class CompactSegmentsTest } } - dataSources = DataSourcesSnapshot - .fromUsedSegments(segments, ImmutableMap.of()) - .getUsedSegmentsTimelinesPerDataSource(); - + dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); leaderClient.start(); @@ -456,10 +450,10 @@ public class CompactSegmentsTest // 3 intervals, 120 byte, 12 segments already compacted before the run for (int compactionRunCount = 0; compactionRunCount < 8; compactionRunCount++) { // Do a cycle of auto compaction which creates one compaction task - final CoordinatorStats stats = doCompactSegments(compactSegments); + final CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( 1, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); verifySnapshot( @@ -482,10 +476,10 @@ public class CompactSegmentsTest } // Test that stats does not change (and is still correct) when auto compaction runs with everything is fully compacted - final CoordinatorStats stats = doCompactSegments(compactSegments); + final CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( 0, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); verifySnapshot( compactSegments, @@ -521,10 +515,10 @@ public class CompactSegmentsTest doCompactionAndAssertCompactSegmentStatistics(compactSegments, compactionRunCount); } // Test that stats does not change (and is still correct) when auto compaction runs with everything is fully compacted - final CoordinatorStats stats = doCompactSegments(compactSegments); + final CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( 0, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); for (int i = 0; i < 3; i++) { verifySnapshot( @@ -544,7 +538,8 @@ public class CompactSegmentsTest } // Deactivate one datasource (datasource 0 no longer exist in timeline) - dataSources.remove(DATA_SOURCE_PREFIX + 0); + dataSources.getUsedSegmentsTimelinesPerDataSource() + .remove(DATA_SOURCE_PREFIX + 0); // Test run auto compaction with one datasource deactivated // Snapshot should not contain deactivated datasource @@ -599,10 +594,7 @@ public class CompactSegmentsTest } } - dataSources = DataSourcesSnapshot - .fromUsedSegments(segments, ImmutableMap.of()) - .getUsedSegmentsTimelinesPerDataSource(); - + dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER); leaderClient.start(); @@ -616,10 +608,10 @@ public class CompactSegmentsTest // 3 intervals, 1200 byte (each segment is 100 bytes), 12 segments will be skipped by auto compaction for (int compactionRunCount = 0; compactionRunCount < 8; compactionRunCount++) { // Do a cycle of auto compaction which creates one compaction task - final CoordinatorStats stats = doCompactSegments(compactSegments); + final CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( 1, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); verifySnapshot( @@ -640,10 +632,10 @@ public class CompactSegmentsTest } // Test that stats does not change (and is still correct) when auto compaction runs with everything is fully compacted - final CoordinatorStats stats = doCompactSegments(compactSegments); + final CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( 0, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); verifySnapshot( compactSegments, @@ -670,10 +662,10 @@ public class CompactSegmentsTest final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); - final CoordinatorStats stats = doCompactSegments(compactSegments, 3); - Assert.assertEquals(3, stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT)); - Assert.assertEquals(3, stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT)); - Assert.assertEquals(3, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)); + final CoordinatorRunStats stats = doCompactSegments(compactSegments, 3); + Assert.assertEquals(3, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Compaction.MAX_SLOTS)); + Assert.assertEquals(3, stats.get(Stats.Compaction.SUBMITTED_TASKS)); } @Test @@ -685,10 +677,10 @@ public class CompactSegmentsTest leaderClient.start(); final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); - final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); - Assert.assertEquals(maxCompactionSlot, stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT)); - Assert.assertEquals(maxCompactionSlot, stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT)); - Assert.assertEquals(maxCompactionSlot, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)); + final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); + Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); + Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.MAX_SLOTS)); + Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.SUBMITTED_TASKS)); } @Test @@ -700,10 +692,10 @@ public class CompactSegmentsTest leaderClient.start(); final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); - final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); - Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT)); - Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT)); - Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)); + final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); + Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); + Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.MAX_SLOTS)); + Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.SUBMITTED_TASKS)); } @Test @@ -1269,10 +1261,10 @@ public class CompactSegmentsTest final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient); final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); - final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4); - Assert.assertEquals(4, stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT)); - Assert.assertEquals(4, stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT)); - Assert.assertEquals(2, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)); + final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4); + Assert.assertEquals(4, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); + Assert.assertEquals(4, stats.get(Stats.Compaction.MAX_SLOTS)); + Assert.assertEquals(2, stats.get(Stats.Compaction.SUBMITTED_TASKS)); } @Test @@ -1302,8 +1294,8 @@ public class CompactSegmentsTest // Verify that locked intervals are skipped and only one compaction task // is submitted for dataSource_0 CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); - final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4); - Assert.assertEquals(1, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)); + final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4); + Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); Assert.assertEquals(1, leaderClient.submittedCompactionTasks.size()); final ClientCompactionTaskQuery compactionTask = leaderClient.submittedCompactionTasks.get(0); @@ -1535,8 +1527,8 @@ public class CompactSegmentsTest // Verify that no locked intervals are skipped CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient); int maxTaskSlots = partitionsSpec instanceof SingleDimensionPartitionsSpec ? 5 : 3; - final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(1), maxTaskSlots); - Assert.assertEquals(3, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)); + final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(1), maxTaskSlots); + Assert.assertEquals(3, stats.get(Stats.Compaction.SUBMITTED_TASKS)); Assert.assertEquals(3, leaderClient.submittedCompactionTasks.size()); leaderClient.submittedCompactionTasks.forEach(task -> { System.out.println(task.getDataSource() + " : " + task.getIoConfig().getInputSpec().getInterval()); @@ -1592,10 +1584,7 @@ public class CompactSegmentsTest 10L ) ); - dataSources = DataSourcesSnapshot - .fromUsedSegments(segments, ImmutableMap.of()) - .getUsedSegmentsTimelinesPerDataSource(); - + dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); @@ -1690,10 +1679,7 @@ public class CompactSegmentsTest 10L ) ); - dataSources = DataSourcesSnapshot - .fromUsedSegments(segments, ImmutableMap.of()) - .getUsedSegmentsTimelinesPerDataSource(); - + dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class); final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient); @@ -1917,10 +1903,10 @@ public class CompactSegmentsTest { for (int dataSourceIndex = 0; dataSourceIndex < 3; dataSourceIndex++) { // One compaction task triggered - final CoordinatorStats stats = doCompactSegments(compactSegments); + final CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( 1, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); // Note: Subsequent compaction run after the dataSource was compacted will show different numbers than // on the run it was compacted. For example, in a compaction run, if a dataSource had 4 segments compacted, @@ -1934,14 +1920,14 @@ public class CompactSegmentsTest compactSegments, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, - TOTAL_BYTE_PER_DATASOURCE - 40 * (compactionRunCount + 1), - 40 * (compactionRunCount + 1), + TOTAL_BYTE_PER_DATASOURCE - 40L * (compactionRunCount + 1), + 40L * (compactionRunCount + 1), 40, TOTAL_INTERVAL_PER_DATASOURCE - (compactionRunCount + 1), (compactionRunCount + 1), 1, - TOTAL_SEGMENT_PER_DATASOURCE - 4 * (compactionRunCount + 1), - 2 * (compactionRunCount + 1), + TOTAL_SEGMENT_PER_DATASOURCE - 4L * (compactionRunCount + 1), + 2L * (compactionRunCount + 1), 4 ); } else { @@ -1949,14 +1935,14 @@ public class CompactSegmentsTest compactSegments, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, - TOTAL_BYTE_PER_DATASOURCE - 40 * (compactionRunCount + 1), - 40 * (compactionRunCount + 1), + TOTAL_BYTE_PER_DATASOURCE - 40L * (compactionRunCount + 1), + 40L * (compactionRunCount + 1), 40, TOTAL_INTERVAL_PER_DATASOURCE - (compactionRunCount + 1), (compactionRunCount + 1), 1, - TOTAL_SEGMENT_PER_DATASOURCE - 4 * (compactionRunCount + 1), - 2 * compactionRunCount + 4, + TOTAL_SEGMENT_PER_DATASOURCE - 4L * (compactionRunCount + 1), + 2L * compactionRunCount + 4, 4 ); } @@ -1968,39 +1954,39 @@ public class CompactSegmentsTest compactSegments, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, - TOTAL_BYTE_PER_DATASOURCE - 40 * compactionRunCount, - 40 * compactionRunCount, + TOTAL_BYTE_PER_DATASOURCE - 40L * compactionRunCount, + 40L * compactionRunCount, 40, TOTAL_INTERVAL_PER_DATASOURCE - compactionRunCount, compactionRunCount, 1, - TOTAL_SEGMENT_PER_DATASOURCE - 4 * compactionRunCount, - 2 * compactionRunCount, + TOTAL_SEGMENT_PER_DATASOURCE - 4L * compactionRunCount, + 2L * compactionRunCount, 4 ); } } } - private CoordinatorStats doCompactSegments(CompactSegments compactSegments) + private CoordinatorRunStats doCompactSegments(CompactSegments compactSegments) { return doCompactSegments(compactSegments, (Integer) null); } - private CoordinatorStats doCompactSegments(CompactSegments compactSegments, @Nullable Integer numCompactionTaskSlots) + private CoordinatorRunStats doCompactSegments(CompactSegments compactSegments, @Nullable Integer numCompactionTaskSlots) { return doCompactSegments(compactSegments, createCompactionConfigs(), numCompactionTaskSlots); } - private CoordinatorStats doCompactSegments( + private void doCompactSegments( CompactSegments compactSegments, List compactionConfigs ) { - return doCompactSegments(compactSegments, compactionConfigs, null); + doCompactSegments(compactSegments, compactionConfigs, null); } - private CoordinatorStats doCompactSegments( + private CoordinatorRunStats doCompactSegments( CompactSegments compactSegments, List compactionConfigs, @Nullable Integer numCompactionTaskSlots @@ -2009,20 +1995,20 @@ public class CompactSegmentsTest return doCompactSegments(compactSegments, compactionConfigs, numCompactionTaskSlots, false); } - private CoordinatorStats doCompactSegments( + private CoordinatorRunStats doCompactSegments( CompactSegments compactSegments, List compactionConfigs, @Nullable Integer numCompactionTaskSlots, boolean useAutoScaleSlots ) { - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withUsedSegmentsTimelinesPerDataSourceInTest(dataSources) + DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc()) + .withSnapshotOfDataSourcesWithAllUsedSegments(dataSources) .withCompactionConfig( new CoordinatorCompactionConfig( compactionConfigs, - numCompactionTaskSlots == null ? null : 1., // 100% when numCompactionTaskSlots is not null + numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null numCompactionTaskSlots, useAutoScaleSlots ) @@ -2040,35 +2026,37 @@ public class CompactSegmentsTest ) { for (int i = 0; i < 3; i++) { - final CoordinatorStats stats = doCompactSegments(compactSegments); + final CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( expectedCompactTaskCount, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) + ); + + // If expectedRemainingSegments is positive, we count the number of datasources + // which have that many segments waiting for compaction. Otherwise, we count + // all the datasources in the coordinator stats + final AtomicInteger numDatasources = new AtomicInteger(); + stats.forEachStat( + (dimensionValues, stat, value) -> { + if (stat.equals(Stats.Compaction.PENDING_BYTES) + && (expectedRemainingSegments <= 0 || value == expectedRemainingSegments)) { + numDatasources.incrementAndGet(); + } + } ); - // One of dataSource is compacted if (expectedRemainingSegments > 0) { - // If expectedRemainingSegments is positive, we check how many dataSources have the segments waiting for - // compaction. - long numDataSourceOfExpectedRemainingSegments = stats - .getDataSources(CompactSegments.TOTAL_SIZE_OF_SEGMENTS_AWAITING) - .stream() - .mapToLong(ds -> stats.getDataSourceStat(CompactSegments.TOTAL_SIZE_OF_SEGMENTS_AWAITING, ds)) - .filter(stat -> stat == expectedRemainingSegments) - .count(); - Assert.assertEquals(i + 1, numDataSourceOfExpectedRemainingSegments); + Assert.assertEquals(i + 1, numDatasources.get()); } else { - // Otherwise, we check how many dataSources are in the coordinator stats. - Assert.assertEquals( - 2 - i, - stats.getDataSources(CompactSegments.TOTAL_SIZE_OF_SEGMENTS_AWAITING).size() - ); + Assert.assertEquals(2 - i, numDatasources.get()); } } + final Map dataSourceToTimeline + = dataSources.getUsedSegmentsTimelinesPerDataSource(); for (int i = 0; i < 3; i++) { final String dataSource = DATA_SOURCE_PREFIX + i; - List> holders = dataSources.get(dataSource).lookup(expectedInterval); + List> holders = dataSourceToTimeline.get(dataSource).lookup(expectedInterval); Assert.assertEquals(1, holders.size()); List> chunks = Lists.newArrayList(holders.get(0).getObject()); Assert.assertEquals(2, chunks.size()); @@ -2083,10 +2071,12 @@ public class CompactSegmentsTest private void assertLastSegmentNotCompacted(CompactSegments compactSegments) { // Segments of the latest interval should not be compacted + final Map dataSourceToTimeline + = dataSources.getUsedSegmentsTimelinesPerDataSource(); for (int i = 0; i < 3; i++) { final String dataSource = DATA_SOURCE_PREFIX + i; final Interval interval = Intervals.of(StringUtils.format("2017-01-09T12:00:00/2017-01-10")); - List> holders = dataSources.get(dataSource).lookup(interval); + List> holders = dataSourceToTimeline.get(dataSource).lookup(interval); Assert.assertEquals(1, holders.size()); for (TimelineObjectHolder holder : holders) { List> chunks = Lists.newArrayList(holder.getObject()); @@ -2103,10 +2093,10 @@ public class CompactSegmentsTest final String dataSource = DATA_SOURCE_PREFIX + 0; addMoreData(dataSource, 9); - CoordinatorStats stats = doCompactSegments(compactSegments); + CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( 1, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); addMoreData(dataSource, 10); @@ -2114,21 +2104,23 @@ public class CompactSegmentsTest stats = doCompactSegments(compactSegments); Assert.assertEquals( 1, - stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) + stats.get(Stats.Compaction.SUBMITTED_TASKS) ); } private void addMoreData(String dataSource, int day) { + final SegmentTimeline timeline + = dataSources.getUsedSegmentsTimelinesPerDataSource().get(dataSource); for (int i = 0; i < 2; i++) { DataSegment newSegment = createSegment(dataSource, day, true, i); - dataSources.get(dataSource).add( + timeline.add( newSegment.getInterval(), newSegment.getVersion(), newSegment.getShardSpec().createChunk(newSegment) ); newSegment = createSegment(dataSource, day, false, i); - dataSources.get(dataSource).add( + timeline.add( newSegment.getInterval(), newSegment.getVersion(), newSegment.getShardSpec().createChunk(newSegment) @@ -2277,19 +2269,15 @@ public class CompactSegmentsTest submittedCompactionTasks.add(compactionTaskQuery); final Interval intervalToCompact = compactionTaskQuery.getIoConfig().getInputSpec().getInterval(); - final VersionedIntervalTimeline timeline = dataSources.get( - compactionTaskQuery.getDataSource() - ); + final SegmentTimeline timeline = dataSources.getUsedSegmentsTimelinesPerDataSource() + .get(compactionTaskQuery.getDataSource()); final List segments = timeline.lookup(intervalToCompact) .stream() .flatMap(holder -> Streams.sequentialStreamFrom(holder.getObject())) .map(PartitionChunk::getObject) .collect(Collectors.toList()); - compactSegments( - timeline, - segments, - compactionTaskQuery - ); + + compactSegments(timeline, segments, compactionTaskQuery); return createStringFullResponseHolder(jsonMapper.writeValueAsString(ImmutableMap.of("task", taskQuery.getId()))); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetricsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetricsTest.java deleted file mode 100644 index 201c0b7f7de..00000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetricsTest.java +++ /dev/null @@ -1,124 +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.druid.server.coordinator.duty; - -import com.google.common.collect.ImmutableMap; -import it.unimi.dsi.fastutil.objects.Object2IntMaps; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.emitter.service.ServiceEventBuilder; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidCluster; -import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.MockitoJUnitRunner; - -import java.util.List; - -@RunWith(MockitoJUnitRunner.class) -public class EmitClusterStatsAndMetricsTest -{ - @Mock - private ServiceEmitter mockServiceEmitter; - @Mock - private DruidCoordinator mockDruidCoordinator; - @Mock - private DruidCoordinatorRuntimeParams mockDruidCoordinatorRuntimeParams; - @Mock - CoordinatorStats mockCoordinatorStats; - @Mock - DruidCluster mockDruidCluster; - @Mock - MetadataRuleManager mockMetadataRuleManager; - - @Test - public void testRunOnlyEmitStatsForHistoricalDuties() - { - ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(ServiceEventBuilder.class); - Mockito.when(mockDruidCoordinatorRuntimeParams.getEmitter()).thenReturn(mockServiceEmitter); - Mockito.when(mockDruidCoordinatorRuntimeParams.getCoordinatorStats()).thenReturn(mockCoordinatorStats); - Mockito.when(mockDruidCoordinatorRuntimeParams.getDruidCluster()).thenReturn(mockDruidCluster); - Mockito.when(mockDruidCoordinatorRuntimeParams.getDatabaseRuleManager()).thenReturn(mockMetadataRuleManager); - Mockito.when(mockDruidCoordinator.computeNumsUnavailableUsedSegmentsPerDataSource()).thenReturn(Object2IntMaps.emptyMap()); - Mockito.when(mockDruidCoordinator.computeUnderReplicationCountsPerDataSourcePerTier()).thenReturn(ImmutableMap.of()); - CoordinatorDuty duty = new EmitClusterStatsAndMetrics(mockDruidCoordinator, DruidCoordinator.HISTORICAL_MANAGEMENT_DUTIES_DUTY_GROUP, false); - duty.run(mockDruidCoordinatorRuntimeParams); - Mockito.verify(mockServiceEmitter, Mockito.atLeastOnce()).emit(argumentCaptor.capture()); - List emittedEvents = argumentCaptor.getAllValues(); - boolean foundCompactMetric = false; - boolean foundHistoricalDutyMetric = false; - boolean foundSegmentDeletedCount = false; - for (ServiceEventBuilder eventBuilder : emittedEvents) { - ServiceMetricEvent serviceMetricEvent = ((ServiceMetricEvent) eventBuilder.build("x", "x")); - String metric = serviceMetricEvent.getMetric(); - if ("segment/overShadowed/count".equals(metric)) { - foundHistoricalDutyMetric = true; - } else if ("compact/task/count".equals(metric)) { - foundCompactMetric = true; - } else if ("segment/deleted/count".equals(metric)) { - foundSegmentDeletedCount = true; - continue; - } - String dutyGroup = (String) serviceMetricEvent.getUserDims().get("dutyGroup"); - Assert.assertNotNull(dutyGroup); - Assert.assertEquals(DruidCoordinator.HISTORICAL_MANAGEMENT_DUTIES_DUTY_GROUP, dutyGroup); - } - Assert.assertTrue(foundHistoricalDutyMetric); - Assert.assertFalse(foundCompactMetric); - Assert.assertTrue(foundSegmentDeletedCount); - } - - @Test - public void testRunEmitStatsForCompactionWhenHaveCompactSegmentDuty() - { - String groupName = "blah"; - ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(ServiceEventBuilder.class); - Mockito.when(mockDruidCoordinatorRuntimeParams.getEmitter()).thenReturn(mockServiceEmitter); - Mockito.when(mockDruidCoordinatorRuntimeParams.getCoordinatorStats()).thenReturn(mockCoordinatorStats); - Mockito.when(mockDruidCoordinatorRuntimeParams.getDruidCluster()).thenReturn(mockDruidCluster); - CoordinatorDuty duty = new EmitClusterStatsAndMetrics(mockDruidCoordinator, groupName, true); - duty.run(mockDruidCoordinatorRuntimeParams); - Mockito.verify(mockServiceEmitter, Mockito.atLeastOnce()).emit(argumentCaptor.capture()); - List emittedEvents = argumentCaptor.getAllValues(); - boolean foundCompactMetric = false; - boolean foundHistoricalDutyMetric = false; - for (ServiceEventBuilder eventBuilder : emittedEvents) { - ServiceMetricEvent serviceMetricEvent = ((ServiceMetricEvent) eventBuilder.build("x", "x")); - String metric = serviceMetricEvent.getMetric(); - if ("segment/overShadowed/count".equals(metric)) { - foundHistoricalDutyMetric = true; - } else if ("compact/task/count".equals(metric)) { - foundCompactMetric = true; - } - String dutyGroup = (String) serviceMetricEvent.getUserDims().get("dutyGroup"); - Assert.assertNotNull(dutyGroup); - Assert.assertEquals(groupName, dutyGroup); - } - Assert.assertFalse(foundHistoricalDutyMetric); - Assert.assertTrue(foundCompactMetric); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegmentsTest.java deleted file mode 100644 index 6db88b86a27..00000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegmentsTest.java +++ /dev/null @@ -1,141 +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.druid.server.coordinator.duty; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import junitparams.JUnitParamsRunner; -import junitparams.Parameters; -import org.apache.druid.client.ImmutableDruidDataSource; -import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidCluster; -import org.apache.druid.server.coordinator.DruidClusterBuilder; -import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.LoadQueuePeon; -import org.apache.druid.server.coordinator.RunRulesTest; -import org.apache.druid.server.coordinator.ServerHolder; -import org.apache.druid.timeline.DataSegment; -import org.easymock.EasyMock; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.junit.Test; -import org.junit.runner.RunWith; - -import java.util.List; - -@RunWith(JUnitParamsRunner.class) -public class MarkAsUnusedOvershadowedSegmentsTest -{ - MarkAsUnusedOvershadowedSegments markAsUnusedOvershadowedSegments; - DruidCoordinator coordinator = EasyMock.createStrictMock(DruidCoordinator.class); - private List usedSegments; - DateTime start = DateTimes.of("2012-01-01"); - DruidCluster druidCluster; - private LoadQueuePeon mockPeon = EasyMock.createMock(LoadQueuePeon.class); - private ImmutableDruidServer druidServer = EasyMock.createMock(ImmutableDruidServer.class); - private ImmutableDruidDataSource druidDataSource = EasyMock.createMock(ImmutableDruidDataSource.class); - private DataSegment segmentV0 = new DataSegment.Builder().dataSource("test") - .interval(new Interval(start, start.plusHours(1))) - .version("0") - .size(0) - .build(); - private DataSegment segmentV1 = new DataSegment.Builder().dataSource("test") - .interval(new Interval(start, start.plusHours(1))) - .version("1") - .size(0) - .build(); - private DataSegment segmentV2 = new DataSegment.Builder().dataSource("test") - .interval(new Interval(start, start.plusHours(1))) - .version("2") - .size(0) - .build(); - - @Test - @Parameters( - { - "historical", - "broker" - } - ) - public void testRun(String serverTypeString) - { - ServerType serverType = ServerType.fromString(serverTypeString); - - markAsUnusedOvershadowedSegments = - new MarkAsUnusedOvershadowedSegments(coordinator); - usedSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); - - // Dummy values for comparisons in TreeSet - EasyMock.expect(mockPeon.getLoadQueueSize()) - .andReturn(0L) - .anyTimes(); - EasyMock.expect(druidServer.getMaxSize()) - .andReturn(0L) - .anyTimes(); - EasyMock.expect(druidServer.getCurrSize()) - .andReturn(0L) - .anyTimes(); - EasyMock.expect(druidServer.getName()) - .andReturn("") - .anyTimes(); - EasyMock.expect(druidServer.getHost()) - .andReturn("") - .anyTimes(); - EasyMock.expect(druidServer.getTier()) - .andReturn("") - .anyTimes(); - EasyMock.expect(druidServer.getType()) - .andReturn(serverType) - .anyTimes(); - - EasyMock.expect(druidServer.getDataSources()) - .andReturn(ImmutableList.of(druidDataSource)) - .anyTimes(); - EasyMock.expect(druidDataSource.getSegments()) - .andReturn(ImmutableSet.of(segmentV1, segmentV2)) - .anyTimes(); - EasyMock.expect(druidDataSource.getName()).andReturn("test").anyTimes(); - coordinator.markSegmentsAsUnused("test", ImmutableSet.of(segmentV1.getId(), segmentV0.getId())); - EasyMock.expectLastCall(); - EasyMock.replay(mockPeon, coordinator, druidServer, druidDataSource); - - druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("normal", new ServerHolder(druidServer, mockPeon)) - .build(); - - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withUsedSegmentsInTest(usedSegments) - .withCoordinatorStats(new CoordinatorStats()) - .withDruidCluster(druidCluster) - .withDynamicConfigs( - RunRulesTest.COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS - ) - .build(); - markAsUnusedOvershadowedSegments.run(params); - EasyMock.verify(coordinator, druidDataSource, druidServer); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java new file mode 100644 index 00000000000..c8167844563 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java @@ -0,0 +1,104 @@ +/* + * 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.druid.server.coordinator.duty; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; +import org.apache.druid.client.DruidServer; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.timeline.DataSegment; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Test; +import org.junit.runner.RunWith; + +import java.util.Collections; +import java.util.List; + +@RunWith(JUnitParamsRunner.class) +public class MarkOvershadowedSegmentsAsUnusedTest +{ + private final DruidCoordinator coordinator = EasyMock.createStrictMock(DruidCoordinator.class); + + private final DateTime start = DateTimes.of("2012-01-01"); + + private final LoadQueuePeon mockPeon = EasyMock.createMock(LoadQueuePeon.class); + private final ImmutableDruidDataSource druidDataSource = EasyMock.createMock(ImmutableDruidDataSource.class); + private final DataSegment segmentV0 = DataSegment.builder().dataSource("test") + .interval(new Interval(start, start.plusHours(1))) + .version("0") + .size(0) + .build(); + private final DataSegment segmentV1 = segmentV0.withVersion("1"); + private final DataSegment segmentV2 = segmentV0.withVersion("2"); + + @Test + @Parameters({"historical", "broker"}) + public void testRun(String serverTypeString) + { + ServerType serverType = ServerType.fromString(serverTypeString); + + MarkOvershadowedSegmentsAsUnused markOvershadowedSegmentsAsUnused = + new MarkOvershadowedSegmentsAsUnused(coordinator); + final List usedSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); + + // Dummy values for comparisons in TreeSet + EasyMock.expect(mockPeon.getSegmentsInQueue()) + .andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()) + .andReturn(Collections.emptySet()).anyTimes(); + final ImmutableDruidServer druidServer = new DruidServer("", "", "", 0L, serverType, "", 0) + .addDataSegment(segmentV1) + .addDataSegment(segmentV2) + .toImmutableDruidServer(); + + coordinator.markSegmentsAsUnused("test", ImmutableSet.of(segmentV1.getId(), segmentV0.getId())); + EasyMock.expectLastCall(); + EasyMock.replay(mockPeon, coordinator, druidDataSource); + + DruidCluster druidCluster = DruidCluster + .builder() + .addTier("normal", new ServerHolder(druidServer, mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc()) + .withUsedSegmentsInTest(usedSegments) + .withDruidCluster(druidCluster) + .withDynamicConfigs( + CoordinatorDynamicConfig.builder().withMarkSegmentAsUnusedDelayMillis(0).build() + ) + .build(); + markOvershadowedSegmentsAsUnused.run(params); + EasyMock.verify(coordinator, druidDataSource); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java index c2e7a05e7be..e2df83ac9a0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java @@ -80,7 +80,7 @@ public class NewestSegmentFirstPolicyTest private static final String DATA_SOURCE = "dataSource"; private static final long DEFAULT_SEGMENT_SIZE = 1000; private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; - private ObjectMapper mapper = new DefaultObjectMapper(); + private final ObjectMapper mapper = new DefaultObjectMapper(); private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(mapper); @Test @@ -1762,13 +1762,28 @@ public class NewestSegmentFirstPolicyTest segments.add(segment); } - remainingInterval = SegmentCompactionUtil.removeIntervalFromEnd(remainingInterval, segmentInterval); + remainingInterval = removeIntervalFromEnd(remainingInterval, segmentInterval); } } return SegmentTimeline.forSegments(segments); } + /** + * Returns an interval [largeInterval.start - smallInterval.start) given that + * the end of both intervals is the same. + */ + private static Interval removeIntervalFromEnd(Interval largeInterval, Interval smallInterval) + { + Preconditions.checkArgument( + largeInterval.getEnd().equals(smallInterval.getEnd()), + "end should be same. largeInterval[%s] smallInterval[%s]", + largeInterval, + smallInterval + ); + return new Interval(largeInterval.getStart(), smallInterval.getStart()); + } + private DataSourceCompactionConfig createCompactionConfig( long inputSegmentSizeBytes, Period skipOffsetFromLatest, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java new file mode 100644 index 00000000000..abc218d8946 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java @@ -0,0 +1,1256 @@ +/* + * 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.druid.server.coordinator.duty; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.client.DruidServer; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.core.EventMap; +import org.apache.druid.java.util.emitter.service.AlertEvent; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.metadata.MetadataRuleManager; +import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.balancer.CostBalancerStrategy; +import org.apache.druid.server.coordinator.balancer.RandomBalancerStrategy; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; +import org.apache.druid.server.coordinator.rules.ForeverLoadRule; +import org.apache.druid.server.coordinator.rules.IntervalDropRule; +import org.apache.druid.server.coordinator.rules.IntervalLoadRule; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * + */ +public class RunRulesTest +{ + private static final long SERVER_SIZE_10GB = 10L << 30; + private static final String DATASOURCE = "test"; + + private LoadQueuePeon mockPeon; + private RunRules ruleRunner; + private StubServiceEmitter emitter; + private MetadataRuleManager databaseRuleManager; + private SegmentsMetadataManager segmentsMetadataManager; + private SegmentLoadQueueManager loadQueueManager; + private final List usedSegments = + CreateDataSegments.ofDatasource(DATASOURCE) + .forIntervals(24, Granularities.HOUR) + .startingAt("2012-01-01") + .withNumPartitions(1) + .eachOfSizeInMb(1); + + private ListeningExecutorService balancerExecutor; + + @Before + public void setUp() + { + mockPeon = EasyMock.createMock(LoadQueuePeon.class); + emitter = new StubServiceEmitter("coordinator", "host"); + EmittingLogger.registerEmitter(emitter); + databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); + segmentsMetadataManager = EasyMock.createNiceMock(SegmentsMetadataManager.class); + ruleRunner = new RunRules(); + loadQueueManager = new SegmentLoadQueueManager(null, segmentsMetadataManager, null); + balancerExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); + } + + @After + public void tearDown() + { + balancerExecutor.shutdown(); + EasyMock.verify(databaseRuleManager); + } + + /** + * Nodes: + * normal - 2 replicants + * maxNonPrimaryReplicantsToLoad - 10 + * Expect only 34 segments to be loaded despite there being 48 primary + non-primary replicants to load! + */ + @Test + public void testOneTierTwoReplicantsWithStrictReplicantLimit() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Collections.singletonList( + new IntervalLoadRule( + Intervals.of("2012-01-01/2012-01-02"), + ImmutableMap.of("normal", 2) + ) + )).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + // server1 has all the segments already loaded + final DruidServer server1 = createHistorical("server1", "normal"); + usedSegments.forEach(server1::addDataSegment); + + final DruidServer server2 = createHistorical("server2", "normal"); + final DruidCluster druidCluster = DruidCluster + .builder() + .addTier( + "normal", + new ServerHolder(server1.toImmutableDruidServer(), mockPeon), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon) + ).build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withDynamicConfigs( + CoordinatorDynamicConfig + .builder() + .withMaxNonPrimaryReplicantsToLoad(10) + .withSmartSegmentLoading(false) + .build() + ) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + + // There are 24 under-replicated segments, but only 10 replicas are assigned + Assert.assertEquals(10L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "normal", DATASOURCE)); + + EasyMock.verify(mockPeon); + } + + /** + * Nodes: + * normal - 2 replicants + * hot - 2 replicants + * maxNonPrimaryReplicantsToLoad - 48 + * Expect only 72 segments to be loaded despite there being 96 primary + non-primary replicants to load! + */ + @Test + public void testTwoTiersTwoReplicantsWithStrictReplicantLimit() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Collections.singletonList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), + ImmutableMap.of("hot", 2, "normal", 2) + ) + )).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + final DruidServer serverHot1 = createHistorical("serverHot", "hot"); + final DruidServer serverHot2 = createHistorical("serverHot2", "hot"); + usedSegments.forEach(serverHot1::addDataSegment); + + final DruidServer serverNorm1 = createHistorical("serverNorm", "normal"); + final DruidServer serverNorm2 = createHistorical("serverNorm2", "normal"); + + DruidCluster druidCluster = DruidCluster + .builder() + .addTier( + "hot", + new ServerHolder(serverHot1.toImmutableDruidServer(), mockPeon), + new ServerHolder(serverHot2.toImmutableDruidServer(), mockPeon) + ) + .addTier( + "normal", + new ServerHolder(serverNorm1.toImmutableDruidServer(), mockPeon), + new ServerHolder(serverNorm2.toImmutableDruidServer(), mockPeon) + ) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withDynamicConfigs( + CoordinatorDynamicConfig.builder() + .withMaxNonPrimaryReplicantsToLoad(10) + .withSmartSegmentLoading(false) + .build() + ) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + + // maxNonPrimaryReplicantsToLoad takes effect on hot tier, but not normal tier + Assert.assertEquals(10L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "hot", DATASOURCE)); + Assert.assertEquals(48L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "normal", DATASOURCE)); + + EasyMock.verify(mockPeon); + } + + /** + * Nodes: + * hot - 1 replicant + * normal - 1 replicant + * cold - 1 replicant + */ + @Test + public void testRunThreeTiersOneReplicant() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T06:00:00.000Z"), + ImmutableMap.of("hot", 1) + ), + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), + ImmutableMap.of("normal", 1) + ), + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), + ImmutableMap.of("cold", 1) + ) + )).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DruidCluster druidCluster = DruidCluster + .builder() + .addTier( + "hot", + new ServerHolder( + createHistorical("serverHot", "hot").toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( + "normal", + new ServerHolder( + createHistorical("serverNorm", "normal").toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( + "cold", + new ServerHolder( + createHistorical("serverCold", "cold").toImmutableDruidServer(), + mockPeon + ) + ) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + + Assert.assertEquals(6L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "hot", DATASOURCE)); + Assert.assertEquals(6L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "normal", DATASOURCE)); + Assert.assertEquals(12L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "cold", DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + EasyMock.verify(mockPeon); + } + + private DruidServer createHistorical(String name, String tier) + { + return new DruidServer(name, name, null, SERVER_SIZE_10GB, ServerType.HISTORICAL, tier, 0); + } + + private ServerHolder createServerHolder(String name, String tier, LoadQueuePeon peon) + { + return new ServerHolder(createHistorical(name, tier).toImmutableDruidServer(), peon); + } + + private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams( + DruidCluster druidCluster, + DataSegment segment + ) + { + return createCoordinatorRuntimeParams(druidCluster, Collections.singletonList(segment)); + } + + private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams(DruidCluster druidCluster) + { + return createCoordinatorRuntimeParams(druidCluster, usedSegments); + } + + private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams( + DruidCluster druidCluster, + List dataSegments + ) + { + return DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc().minusDays(1)) + .withDruidCluster(druidCluster) + .withUsedSegmentsInTest(dataSegments) + .withDatabaseRuleManager(databaseRuleManager); + } + + /** + * Nodes: + * hot - 2 replicants + * cold - 1 replicant + */ + @Test + public void testRunTwoTiersTwoReplicants() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T06:00:00.000Z"), + ImmutableMap.of("hot", 2) + ), + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), + ImmutableMap.of("cold", 1) + ) + ) + ).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DruidCluster druidCluster = DruidCluster + .builder() + .add(createServerHolder("serverHot", "hot", mockPeon)) + .add(createServerHolder("serverHot2", "hot", mockPeon)) + .add(createServerHolder("serverCold", "cold", mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + CoordinatorRunStats stats = runDutyAndGetStats(params); + + Assert.assertEquals(12L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "hot", DATASOURCE)); + Assert.assertEquals(18L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "cold", DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + EasyMock.verify(mockPeon); + } + + /** + * Nodes: + * hot - 1 replicant + * normal - 1 replicant + */ + @Test + public void testRunTwoTiersWithExistingSegments() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), + ImmutableMap.of("hot", 1) + ), + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), + ImmutableMap.of("normal", 1) + ) + ) + ).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DruidServer normServer = createHistorical("serverNorm", "normal"); + for (DataSegment segment : usedSegments) { + normServer.addDataSegment(segment); + } + + DruidCluster druidCluster = DruidCluster + .builder() + .add(createServerHolder("serverHot", "hot", mockPeon)) + .add(new ServerHolder(normServer.toImmutableDruidServer(), mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + + Assert.assertEquals(12L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "hot", DATASOURCE)); + Assert.assertEquals(0L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "normal", DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + EasyMock.verify(mockPeon); + } + + @Test + public void testRunTwoTiersTierDoesNotExist() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), + ImmutableMap.of("hot", 1) + ), + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), + ImmutableMap.of("normal", 1) + ) + ) + ).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DruidCluster druidCluster = DruidCluster + .builder() + .add(createServerHolder("serverNorm", "normal", mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + runDutyAndGetStats(params); + + EasyMock.verify(mockPeon); + } + + @Test + public void testRunRuleDoesNotExist() + { + + EasyMock + .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) + .andReturn( + Collections.singletonList( + new IntervalLoadRule( + Intervals.of("2012-01-02T00:00:00.000Z/2012-01-03T00:00:00.000Z"), + ImmutableMap.of("normal", 1) + ) + ) + ) + .atLeastOnce(); + + EasyMock.expect(mockPeon.getSegmentsInQueue()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.replay(databaseRuleManager, mockPeon); + + DruidCluster druidCluster = DruidCluster + .builder() + .add(createServerHolder("serverNorm", "normal", mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster).build(); + + runDutyAndGetStats(params); + + final List events = emitter.getEvents(); + Assert.assertEquals(1, events.size()); + + AlertEvent alertEvent = (AlertEvent) events.get(0); + EventMap eventMap = alertEvent.toMap(); + Assert.assertEquals("Unable to find matching rules!", eventMap.get("description")); + + Map dataMap = alertEvent.getDataMap(); + Assert.assertEquals(usedSegments.size(), dataMap.get("segmentsWithMissingRulesCount")); + + EasyMock.verify(mockPeon); + } + + @Test + public void testDropRemove() + { + mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(segmentsMetadataManager.markSegmentAsUnused(EasyMock.anyObject())) + .andReturn(true).anyTimes(); + EasyMock.replay(segmentsMetadataManager); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), + ImmutableMap.of("normal", 1) + ), + new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) + ) + ).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DruidServer server = createHistorical("serverNorm", "normal"); + for (DataSegment segment : usedSegments) { + server.addDataSegment(segment); + } + + DruidCluster druidCluster = DruidCluster + .builder() + .addTier("normal", new ServerHolder(server.toImmutableDruidServer(), mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + Assert.assertEquals(12L, stats.getDataSourceStat(Stats.Segments.DELETED, DATASOURCE)); + } + + @Test + public void testDropTooManyInSameTier() + { + mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), + ImmutableMap.of("normal", 1) + ), + new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) + ) + ).atLeastOnce(); + EasyMock.replay(databaseRuleManager, segmentsMetadataManager); + + DruidServer server1 = createHistorical("serverNorm", "normal"); + server1.addDataSegment(usedSegments.get(0)); + + DruidServer server2 = createHistorical("serverNorm2", "normal"); + for (DataSegment segment : usedSegments) { + server2.addDataSegment(segment); + } + + DruidCluster druidCluster = DruidCluster + .builder() + .addTier( + "normal", + new ServerHolder(server1.toImmutableDruidServer(), mockPeon), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon) + ) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withDynamicConfigs( + CoordinatorDynamicConfig + .builder() + .withMarkSegmentAsUnusedDelayMillis(0L) + .build() + ) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, "normal", DATASOURCE)); + Assert.assertEquals(12L, stats.getDataSourceStat(Stats.Segments.DELETED, DATASOURCE)); + + EasyMock.verify(mockPeon); + } + + @Test + public void testDropTooManyInDifferentTiers() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), + ImmutableMap.of("hot", 1) + ), + new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) + ) + ).atLeastOnce(); + EasyMock.replay(databaseRuleManager, segmentsMetadataManager); + + DruidServer server1 = createHistorical("server1", "hot"); + server1.addDataSegment(usedSegments.get(0)); + DruidServer server2 = createHistorical("serverNorm2", "normal"); + for (DataSegment segment : usedSegments) { + server2.addDataSegment(segment); + } + + DruidCluster druidCluster = DruidCluster + .builder() + .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + .addTier("normal", new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, "normal", DATASOURCE)); + Assert.assertEquals(12L, stats.getDataSourceStat(Stats.Segments.DELETED, DATASOURCE)); + + EasyMock.verify(mockPeon); + } + + @Test + public void testDontDropInDifferentTiers() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Lists.newArrayList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), + ImmutableMap.of("hot", 1) + ), + new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) + ) + ).atLeastOnce(); + EasyMock.replay(databaseRuleManager, segmentsMetadataManager); + + DruidServer server1 = createHistorical("server1", "hot"); + DruidServer server2 = createHistorical("serverNorm2", "normal"); + for (DataSegment segment : usedSegments) { + server2.addDataSegment(segment); + } + + DruidCluster druidCluster = DruidCluster + .builder() + .add(new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + .add(new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + Assert.assertEquals(12L, stats.getDataSourceStat(Stats.Segments.DELETED, DATASOURCE)); + + EasyMock.verify(mockPeon); + } + + @Test + public void testDropServerActuallyServesSegment() + { + EasyMock + .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) + .andReturn( + Collections.singletonList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T01:00:00.000Z"), + ImmutableMap.of("normal", 0) + ) + ) + ) + .atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DruidServer server1 = createHistorical("server1", "normal"); + server1.addDataSegment(usedSegments.get(0)); + DruidServer server2 = createHistorical("serverNorm2", "normal"); + server2.addDataSegment(usedSegments.get(1)); + DruidServer server3 = createHistorical("serverNorm3", "normal"); + server3.addDataSegment(usedSegments.get(1)); + server3.addDataSegment(usedSegments.get(2)); + + mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + LoadQueuePeon anotherMockPeon = EasyMock.createMock(LoadQueuePeon.class); + EasyMock.expect(anotherMockPeon.getSegmentsMarkedToDrop()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(anotherMockPeon.getSegmentsInQueue()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(anotherMockPeon.getSegmentsToLoad()).andReturn(Collections.emptySet()).anyTimes(); + + EasyMock.replay(anotherMockPeon); + + DruidCluster druidCluster = DruidCluster + .builder() + .addTier( + "normal", + new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false), + new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false), + new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false) + ) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, "normal", DATASOURCE)); + + EasyMock.verify(mockPeon); + EasyMock.verify(anotherMockPeon); + } + + /** + * Nodes: + * hot - 2 replicants + */ + @Test + public void testNoThrottleWhenSegmentNotLoadedInTier() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock + .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) + .andReturn( + Collections.singletonList( + new IntervalLoadRule( + Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"), + ImmutableMap.of("hot", 2) + ) + ) + ) + .atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DruidCluster druidCluster = DruidCluster + .builder() + .addTier( + "hot", + new ServerHolder( + createHistorical("serverHot", "hot").toImmutableDruidServer(), + mockPeon + ), + new ServerHolder( + createHistorical("serverHot2", "hot").toImmutableDruidServer(), + mockPeon + ) + ) + .build(); + + final CostBalancerStrategy balancerStrategy = new CostBalancerStrategy(balancerExecutor); + DruidCoordinatorRuntimeParams params = + createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(balancerStrategy) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + Assert.assertEquals(48L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "hot", DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + DataSegment overFlowSegment = new DataSegment( + "test", + Intervals.of("2012-02-01/2012-02-02"), + DateTimes.nowUtc().toString(), + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + NoneShardSpec.instance(), + 1, + 0 + ); + + stats = runDutyAndGetStats( + createCoordinatorRuntimeParams(druidCluster) + .withUsedSegmentsInTest(overFlowSegment) + .withBalancerStrategy(balancerStrategy) + .withSegmentAssignerUsing(loadQueueManager) + .build() + ); + + Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "hot", DATASOURCE)); + + EasyMock.verify(mockPeon); + } + + /** + * Nodes: + * hot - nothing loaded + * _default_tier - 1 segment loaded + */ + @Test + public void testReplicantThrottleAcrossTiers() + { + EasyMock.expect(segmentsMetadataManager.markSegmentAsUnused(EasyMock.anyObject())) + .andReturn(true).anyTimes(); + EasyMock.replay(segmentsMetadataManager); + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock + .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) + .andReturn( + Collections.singletonList( + new IntervalLoadRule( + Intervals.of("2012-01-01/2013-01-01"), + ImmutableMap.of("hot", 1, DruidServer.DEFAULT_TIER, 1) + ) + ) + ) + .atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + final DruidCluster druidCluster = DruidCluster + .builder() + .addTier( + "hot", + new ServerHolder( + createHistorical("serverHot", "hot").toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( + DruidServer.DEFAULT_TIER, + new ServerHolder( + createHistorical("serverNorm", "normal").toImmutableDruidServer(), + mockPeon + ) + ) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withReplicationThrottleLimit(7).build()) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + Assert.assertEquals(24L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "hot", DATASOURCE)); + Assert.assertEquals(24L, stats.getSegmentStat(Stats.Segments.ASSIGNED, DruidServer.DEFAULT_TIER, DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + EasyMock.verify(mockPeon); + } + + @Test + public void testDropReplicantThrottle() + { + mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock + .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) + .andReturn( + Collections.singletonList( + new IntervalLoadRule( + Intervals.of("2012-01-01/2013-01-02"), + ImmutableMap.of("normal", 1) + ) + ) + ) + .atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DataSegment overFlowSegment = new DataSegment( + "test", + Intervals.of("2012-02-01/2012-02-02"), + DateTimes.nowUtc().toString(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + NoneShardSpec.instance(), + 1, + 0 + ); + List longerUsedSegments = Lists.newArrayList(usedSegments); + longerUsedSegments.add(overFlowSegment); + + DruidServer server1 = createHistorical("serverNorm1", "normal"); + for (DataSegment segment : longerUsedSegments) { + server1.addDataSegment(segment); + } + DruidServer server2 = createHistorical("serverNorm2", "normal"); + for (DataSegment segment : longerUsedSegments) { + server2.addDataSegment(segment); + } + + DruidCluster druidCluster = + DruidCluster.builder() + .add(new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + .add(new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withUsedSegmentsInTest(longerUsedSegments) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + + // There is no throttling on drop + Assert.assertEquals(25L, stats.getSegmentStat(Stats.Segments.DROPPED, "normal", DATASOURCE)); + EasyMock.verify(mockPeon); + } + + @Test + public void testRulesRunOnNonOvershadowedSegmentsOnly() + { + Set usedSegments = new HashSet<>(); + DataSegment v1 = new DataSegment( + "test", + Intervals.of("2012-01-01/2012-01-02"), + "1", + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 1 + ); + DataSegment v2 = new DataSegment( + "test", + Intervals.of("2012-01-01/2012-01-02"), + "2", + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 1 + ); + usedSegments.add(v1); + usedSegments.add(v2); + + mockPeon.loadSegment(EasyMock.eq(v2), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().once(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Collections.singletonList(new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1)))).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DruidCluster druidCluster = DruidCluster.builder().add( + createServerHolder("serverHot", DruidServer.DEFAULT_TIER, mockPeon) + ).build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withUsedSegmentsInTest(usedSegments) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + Assert.assertEquals(1, stats.getSegmentStat(Stats.Segments.ASSIGNED, DruidServer.DEFAULT_TIER, DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + Assert.assertEquals(2, usedSegments.size()); + Assert.assertEquals(usedSegments, params.getUsedSegments()); + + EasyMock.verify(mockPeon); + } + + /** + * Tier - __default_tier + * Nodes - 2 + * Replicants - 3 + * Random balancer strategy should not assign anything and not get into loop as there are not enough nodes for replication + */ + @Test(timeout = 5000L) + public void testTwoNodesOneTierThreeReplicantsRandomStrategyNotEnoughNodes() + { + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Collections.singletonList( + new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 3)) + )).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DataSegment dataSegment = new DataSegment( + "test", + Intervals.utc(0, 1), + DateTimes.nowUtc().toString(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 1 + ); + + DruidCluster druidCluster = DruidCluster.builder().addTier( + DruidServer.DEFAULT_TIER, + new ServerHolder( + createHistorical("server1", DruidServer.DEFAULT_TIER) + .addDataSegment(dataSegment) + .toImmutableDruidServer(), + mockPeon + ), + new ServerHolder( + createHistorical("server2", DruidServer.DEFAULT_TIER) + .addDataSegment(dataSegment) + .toImmutableDruidServer(), + mockPeon + ) + ).build(); + + DruidCoordinatorRuntimeParams params = + createCoordinatorRuntimeParams(druidCluster, dataSegment) + .withBalancerStrategy(new RandomBalancerStrategy()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + Assert.assertEquals(0L, stats.getSegmentStat(Stats.Segments.ASSIGNED, DruidServer.DEFAULT_TIER, DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + EasyMock.verify(mockPeon); + } + + + /** + * Tier - __default_tier + * Nodes - 1 + * Replicants - 1 + * Random balancer strategy should select the only node + */ + @Test(timeout = 5000L) + public void testOneNodesOneTierOneReplicantRandomStrategyEnoughSpace() + { + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + mockEmptyPeon(); + + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Collections.singletonList( + new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1)) + )).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DataSegment dataSegment = new DataSegment( + "test", + Intervals.utc(0, 1), + DateTimes.nowUtc().toString(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 1 + ); + + DruidCluster druidCluster = DruidCluster.builder().addTier( + DruidServer.DEFAULT_TIER, + new ServerHolder( + createHistorical("server1", DruidServer.DEFAULT_TIER).toImmutableDruidServer(), + mockPeon + ) + ).build(); + + DruidCoordinatorRuntimeParams params = + createCoordinatorRuntimeParams(druidCluster, dataSegment) + .withBalancerStrategy(new RandomBalancerStrategy()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, DruidServer.DEFAULT_TIER, DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + EasyMock.verify(mockPeon); + } + + /** + * Tier - __default_tier + * Nodes - 1 + * Replicants - 1 + * Random balancer strategy should not assign anything as there is not enough space + */ + @Test(timeout = 5000L) + public void testOneNodesOneTierOneReplicantRandomStrategyNotEnoughSpace() + { + mockEmptyPeon(); + int numReplicants = 1; + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Collections.singletonList( + new ForeverLoadRule( + ImmutableMap.of(DruidServer.DEFAULT_TIER, numReplicants) + ) + )).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + final DataSegment dataSegment = new DataSegment( + "test", + Intervals.utc(0, 1), + DateTimes.nowUtc().toString(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 11 + ); + + DruidCluster druidCluster = DruidCluster.builder().addTier( + DruidServer.DEFAULT_TIER, + new ServerHolder( + new DruidServer("server1", "host1", null, 10, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0) + .toImmutableDruidServer(), + mockPeon + ) + ).build(); + + DruidCoordinatorRuntimeParams params = + createCoordinatorRuntimeParams(druidCluster, dataSegment) + .withBalancerStrategy(new RandomBalancerStrategy()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + final RowKey tierRowKey = RowKey.builder().add(Dimension.TIER, DruidServer.DEFAULT_TIER).build(); + Assert.assertEquals( + dataSegment.getSize() * numReplicants, + stats.get(Stats.Tier.REQUIRED_CAPACITY, tierRowKey) + ); + + // Verify that primary assignment failed + Assert.assertEquals(0L, stats.getSegmentStat(Stats.Segments.ASSIGNED, DruidServer.DEFAULT_TIER, DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + EasyMock.verify(mockPeon); + } + + /** + * Tier - __default_tier + * Nodes - 1 + * Replicants - 1 + * Cost balancer strategy should not assign anything as there is not enough space + */ + @Test + public void testOneNodesOneTierOneReplicantCostBalancerStrategyNotEnoughSpace() + { + mockEmptyPeon(); + int numReplicants = 1; + EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( + Collections.singletonList( + new ForeverLoadRule( + ImmutableMap.of(DruidServer.DEFAULT_TIER, numReplicants) + ) + )).atLeastOnce(); + EasyMock.replay(databaseRuleManager); + + DataSegment dataSegment = new DataSegment( + "test", + Intervals.utc(0, 1), + DateTimes.nowUtc().toString(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), + NoneShardSpec.instance(), + IndexIO.CURRENT_VERSION_ID, + 11 + ); + + DruidCluster druidCluster = DruidCluster.builder().addTier( + DruidServer.DEFAULT_TIER, + new ServerHolder( + new DruidServer("server1", "host1", null, 10, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0) + .toImmutableDruidServer(), + mockPeon + ) + ).build(); + + DruidCoordinatorRuntimeParams params = + createCoordinatorRuntimeParams(druidCluster, dataSegment) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + final RowKey tierRowKey = RowKey.builder().add(Dimension.TIER, DruidServer.DEFAULT_TIER).build(); + Assert.assertEquals( + dataSegment.getSize() * numReplicants, + stats.get(Stats.Tier.REQUIRED_CAPACITY, tierRowKey) + ); + Assert.assertEquals(0L, stats.getSegmentStat(Stats.Segments.ASSIGNED, DruidServer.DEFAULT_TIER, DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + EasyMock.verify(mockPeon); + } + + private CoordinatorRunStats runDutyAndGetStats(DruidCoordinatorRuntimeParams params) + { + params = ruleRunner.run(params); + return params.getCoordinatorStats(); + } + + private void mockEmptyPeon() + { + EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsInQueue()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.replay(mockPeon); + } + +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java index f74762c7ad9..8faa4d361d0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java @@ -29,15 +29,16 @@ import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.DruidClusterBuilder; +import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.LoadQueuePeonTester; import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.ForeverLoadRule; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -50,7 +51,6 @@ import org.junit.Test; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Set; @@ -67,18 +67,13 @@ public class UnloadUnusedSegmentsTest private LoadQueuePeonTester indexerPeon; private DataSegment segment1; private DataSegment segment2; - private DataSegment broadcastSegment; - private DataSegment realtimeOnlySegment; private List segments; private List segmentsForRealtime; - private ImmutableDruidDataSource dataSource1; - private ImmutableDruidDataSource dataSource2; - private ImmutableDruidDataSource dataSource2ForRealtime; - private ImmutableDruidDataSource broadcastDatasource; private List dataSources; private List dataSourcesForRealtime; - private Set broadcastDatasourceNames; + private final String broadcastDatasource = "broadcastDatasource"; private MetadataRuleManager databaseRuleManager; + private SegmentLoadQueueManager loadQueueManager; @Before public void setUp() @@ -88,9 +83,8 @@ public class UnloadUnusedSegmentsTest historicalServerTier2 = EasyMock.createMock(ImmutableDruidServer.class); brokerServer = EasyMock.createMock(ImmutableDruidServer.class); indexerServer = EasyMock.createMock(ImmutableDruidServer.class); - segment1 = EasyMock.createMock(DataSegment.class); - segment2 = EasyMock.createMock(DataSegment.class); databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); + loadQueueManager = new SegmentLoadQueueManager(null, null, null); DateTime start1 = DateTimes.of("2012-01-01"); DateTime start2 = DateTimes.of("2012-02-01"); @@ -99,9 +93,9 @@ public class UnloadUnusedSegmentsTest "datasource1", new Interval(start1, start1.plusHours(1)), version.toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), NoneShardSpec.instance(), 0, 11L @@ -110,31 +104,31 @@ public class UnloadUnusedSegmentsTest "datasource2", new Interval(start1, start1.plusHours(1)), version.toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), NoneShardSpec.instance(), 0, 7L ); - realtimeOnlySegment = new DataSegment( + final DataSegment realtimeOnlySegment = new DataSegment( "datasource2", new Interval(start2, start2.plusHours(1)), version.toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), NoneShardSpec.instance(), 0, 7L ); - broadcastSegment = new DataSegment( - "broadcastDatasource", + final DataSegment broadcastSegment = new DataSegment( + broadcastDatasource, new Interval(start1, start1.plusHours(1)), version.toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), NoneShardSpec.instance(), 0, 7L @@ -154,19 +148,18 @@ public class UnloadUnusedSegmentsTest brokerPeon = new LoadQueuePeonTester(); indexerPeon = new LoadQueuePeonTester(); - dataSource1 = new ImmutableDruidDataSource( + final ImmutableDruidDataSource dataSource1 = new ImmutableDruidDataSource( "datasource1", Collections.emptyMap(), Collections.singleton(segment1) ); - dataSource2 = new ImmutableDruidDataSource( + final ImmutableDruidDataSource dataSource2 = new ImmutableDruidDataSource( "datasource2", Collections.emptyMap(), Collections.singleton(segment2) ); - broadcastDatasourceNames = Collections.singleton("broadcastDatasource"); - broadcastDatasource = new ImmutableDruidDataSource( + final ImmutableDruidDataSource broadcastDatasource = new ImmutableDruidDataSource( "broadcastDatasource", Collections.emptyMap(), Collections.singleton(broadcastSegment) @@ -176,7 +169,7 @@ public class UnloadUnusedSegmentsTest // This simulates a task that is ingesting to an existing non-broadcast datasource, with unpublished segments, // while also having a broadcast segment loaded. - dataSource2ForRealtime = new ImmutableDruidDataSource( + final ImmutableDruidDataSource dataSource2ForRealtime = new ImmutableDruidDataSource( "datasource2", Collections.emptyMap(), Collections.singleton(realtimeOnlySegment) @@ -249,11 +242,11 @@ public class UnloadUnusedSegmentsTest // will not appear in the coordinator's view of used segments. Set usedSegments = ImmutableSet.of(segment2); - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() + DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc()) .withDruidCluster( - DruidClusterBuilder - .newBuilder() + DruidCluster + .builder() .addTier( DruidServer.DEFAULT_TIER, new ServerHolder(historicalServer, historicalPeon, false) @@ -262,33 +255,24 @@ public class UnloadUnusedSegmentsTest "tier2", new ServerHolder(historicalServerTier2, historicalTier2Peon, false) ) - .withBrokers( - new ServerHolder(brokerServer, brokerPeon, false) - ) - .withRealtimes( - new ServerHolder(indexerServer, indexerPeon, false) - ) + .addBrokers(new ServerHolder(brokerServer, brokerPeon, false)) + .addRealtimes(new ServerHolder(indexerServer, indexerPeon, false)) .build() ) - .withLoadManagementPeons( - ImmutableMap.of( - "historical", historicalPeon, - "historicalTier2", historicalTier2Peon, - "broker", brokerPeon, - "indexer", indexerPeon - ) - ) .withUsedSegmentsInTest(usedSegments) - .withBroadcastDatasources(broadcastDatasourceNames) + .withBroadcastDatasources(Collections.singleton(broadcastDatasource)) .withDatabaseRuleManager(databaseRuleManager) .build(); - params = new UnloadUnusedSegments().run(params); - CoordinatorStats stats = params.getCoordinatorStats(); + params = new UnloadUnusedSegments(loadQueueManager).run(params); + CoordinatorRunStats stats = params.getCoordinatorStats(); // We drop segment1 and broadcast1 from all servers, realtimeSegment is not dropped by the indexer - Assert.assertEquals(5, stats.getTieredStat("unneededCount", DruidServer.DEFAULT_TIER)); - Assert.assertEquals(2, stats.getTieredStat("unneededCount", "tier2")); + Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.UNNEEDED, DruidServer.DEFAULT_TIER, segment1.getDataSource())); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.UNNEEDED, "tier2", segment1.getDataSource())); + + Assert.assertEquals(3L, stats.getSegmentStat(Stats.Segments.UNNEEDED, DruidServer.DEFAULT_TIER, broadcastDatasource)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.UNNEEDED, "tier2", broadcastDatasource)); } private static void mockDruidServer( @@ -321,14 +305,6 @@ public class UnloadUnusedSegmentsTest private static void mockCoordinator(DruidCoordinator coordinator) { - coordinator.moveSegment( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - ); - EasyMock.expectLastCall().anyTimes(); EasyMock.replay(coordinator); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java new file mode 100644 index 00000000000..c3d3fb0fbe8 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -0,0 +1,382 @@ +/* + * 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.druid.server.coordinator.loading; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.server.ServerTestHelper; +import org.apache.druid.server.coordination.DataSegmentChangeCallback; +import org.apache.druid.server.coordination.DataSegmentChangeHandler; +import org.apache.druid.server.coordination.DataSegmentChangeRequest; +import org.apache.druid.server.coordination.SegmentLoadDropHandler; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; +import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; +import org.apache.druid.timeline.DataSegment; +import org.jboss.netty.buffer.ChannelBuffers; +import org.jboss.netty.handler.codec.http.DefaultHttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.jboss.netty.handler.codec.http.HttpVersion; +import org.joda.time.Duration; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +/** + * + */ +public class HttpLoadQueuePeonTest +{ + private final List segments = + CreateDataSegments.ofDatasource("test") + .forIntervals(1, Granularities.DAY) + .startingAt("2022-01-01") + .withNumPartitions(4) + .eachOfSizeInMb(100); + + private TestHttpClient httpClient; + private HttpLoadQueuePeon httpLoadQueuePeon; + private BlockingExecutorService processingExecutor; + private BlockingExecutorService callbackExecutor; + + private final List processedSegments = new ArrayList<>(); + + @Before + public void setUp() + { + httpClient = new TestHttpClient(); + processingExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-%s"); + callbackExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-cb"); + processedSegments.clear(); + + httpLoadQueuePeon = new HttpLoadQueuePeon( + "http://dummy:4000", + ServerTestHelper.MAPPER, + httpClient, + new TestDruidCoordinatorConfig.Builder() + .withHttpLoadQueuePeonBatchSize(10) + .build(), + new WrappingScheduledExecutorService("HttpLoadQueuePeonTest-%s", processingExecutor, true), + callbackExecutor + ); + httpLoadQueuePeon.start(); + } + + @After + public void tearDown() + { + httpLoadQueuePeon.stop(); + } + + @Test + public void testSimple() + { + httpLoadQueuePeon + .dropSegment(segments.get(0), markSegmentProcessed(segments.get(0))); + httpLoadQueuePeon + .loadSegment(segments.get(1), SegmentAction.LOAD, markSegmentProcessed(segments.get(1))); + httpLoadQueuePeon + .loadSegment(segments.get(2), SegmentAction.REPLICATE, markSegmentProcessed(segments.get(2))); + httpLoadQueuePeon + .loadSegment(segments.get(3), SegmentAction.MOVE_TO, markSegmentProcessed(segments.get(3))); + + // Send requests to server + processingExecutor.finishAllPendingTasks(); + Assert.assertEquals(segments, httpClient.segmentsSentToServer); + + // Verify that all callbacks are executed + callbackExecutor.finishAllPendingTasks(); + Assert.assertEquals(segments, processedSegments); + } + + @Test + public void testLoadDropAfterStop() + { + // Verify that requests sent after stopping the peon fail immediately + httpLoadQueuePeon.stop(); + + final Set failedSegments = new HashSet<>(); + final DataSegment segment1 = segments.get(0); + httpLoadQueuePeon.dropSegment(segment1, success -> { + if (!success) { + failedSegments.add(segment1); + } + }); + final DataSegment segment2 = segments.get(1); + httpLoadQueuePeon.loadSegment(segment2, SegmentAction.MOVE_TO, success -> { + if (!success) { + failedSegments.add(segment2); + } + }); + + Assert.assertTrue(failedSegments.contains(segment1)); + Assert.assertTrue(failedSegments.contains(segment2)); + } + + @Test + public void testPriorityOfSegmentAction() + { + // Shuffle the segments for the same day + final List segmentsDay1 = new ArrayList<>(segments); + Collections.shuffle(segmentsDay1); + + // Assign segments to the actions in their order of priority + // Order: drop, load, replicate, move + final List actions = Arrays.asList( + QueueAction.of(segmentsDay1.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), + QueueAction.of(segmentsDay1.get(1), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.LOAD, null)), + QueueAction.of(segmentsDay1.get(2), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.REPLICATE, null)), + QueueAction.of(segmentsDay1.get(3), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.MOVE_TO, null)) + ); + + // Queue the actions on the peon in a random order + Collections.shuffle(actions); + actions.forEach(QueueAction::invoke); + + // Send one batch of requests to the server + processingExecutor.finishAllPendingTasks(); + + // Verify that all segments are sent to the server in the expected order + Assert.assertEquals(segmentsDay1, httpClient.segmentsSentToServer); + } + + @Test + public void testPriorityOfSegmentInterval() + { + // Create 8 segments (4 x 2days) and shuffle them + final List segmentsDay1 = new ArrayList<>(segments); + Collections.shuffle(segmentsDay1); + + final List segmentsDay2 = new ArrayList<>( + CreateDataSegments.ofDatasource("test") + .forIntervals(1, Granularities.DAY) + .startingAt("2022-01-02") + .withNumPartitions(4) + .eachOfSizeInMb(100) + ); + Collections.shuffle(segmentsDay2); + + // Assign segments to the actions in their order of priority + // Priority order: action (drop, priorityLoad, etc), then interval (new then old) + List actions = Arrays.asList( + QueueAction.of(segmentsDay2.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), + QueueAction.of(segmentsDay1.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), + QueueAction.of(segmentsDay2.get(1), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.LOAD, null)), + QueueAction.of(segmentsDay1.get(1), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.LOAD, null)), + QueueAction.of(segmentsDay2.get(2), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.REPLICATE, null)), + QueueAction.of(segmentsDay1.get(2), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.REPLICATE, null)), + QueueAction.of(segmentsDay2.get(3), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.MOVE_TO, null)), + QueueAction.of(segmentsDay1.get(3), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.MOVE_TO, null)) + ); + final List expectedSegmentOrder = + actions.stream().map(a -> a.segment).collect(Collectors.toList()); + + // Queue the actions on the peon in a random order + Collections.shuffle(actions); + actions.forEach(QueueAction::invoke); + + // Send one batch of requests to the server + processingExecutor.finishNextPendingTask(); + + // Verify that all segments are sent to the server in the expected order + Assert.assertEquals(expectedSegmentOrder, httpClient.segmentsSentToServer); + } + + @Test + public void testCancelLoad() + { + final DataSegment segment = segments.get(0); + httpLoadQueuePeon.loadSegment(segment, SegmentAction.REPLICATE, markSegmentProcessed(segment)); + Assert.assertEquals(1, httpLoadQueuePeon.getSegmentsToLoad().size()); + + boolean cancelled = httpLoadQueuePeon.cancelOperation(segment); + Assert.assertTrue(cancelled); + Assert.assertEquals(0, httpLoadQueuePeon.getSegmentsToLoad().size()); + + Assert.assertTrue(processedSegments.isEmpty()); + } + + @Test + public void testCancelDrop() + { + final DataSegment segment = segments.get(0); + httpLoadQueuePeon.dropSegment(segment, markSegmentProcessed(segment)); + Assert.assertEquals(1, httpLoadQueuePeon.getSegmentsToDrop().size()); + + boolean cancelled = httpLoadQueuePeon.cancelOperation(segment); + Assert.assertTrue(cancelled); + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToDrop().isEmpty()); + + Assert.assertTrue(processedSegments.isEmpty()); + } + + @Test + public void testCannotCancelRequestSentToServer() + { + final DataSegment segment = segments.get(0); + httpLoadQueuePeon.loadSegment(segment, SegmentAction.REPLICATE, markSegmentProcessed(segment)); + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().contains(segment)); + + // Send the request to the server + processingExecutor.finishNextPendingTask(); + Assert.assertTrue(httpClient.segmentsSentToServer.contains(segment)); + + // Segment is still in queue but operation cannot be cancelled + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().contains(segment)); + boolean cancelled = httpLoadQueuePeon.cancelOperation(segment); + Assert.assertFalse(cancelled); + + // Handle response from server + processingExecutor.finishNextPendingTask(); + + // Segment has been removed from queue + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().isEmpty()); + cancelled = httpLoadQueuePeon.cancelOperation(segment); + Assert.assertFalse(cancelled); + + // Execute callbacks and verify segment is fully processed + callbackExecutor.finishAllPendingTasks(); + Assert.assertTrue(processedSegments.contains(segment)); + } + + @Test + public void testCannotCancelOperationMultipleTimes() + { + final DataSegment segment = segments.get(0); + httpLoadQueuePeon.loadSegment(segment, SegmentAction.REPLICATE, markSegmentProcessed(segment)); + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().contains(segment)); + + Assert.assertTrue(httpLoadQueuePeon.cancelOperation(segment)); + Assert.assertFalse(httpLoadQueuePeon.cancelOperation(segment)); + } + + private LoadPeonCallback markSegmentProcessed(DataSegment segment) + { + return success -> processedSegments.add(segment); + } + + private static class TestHttpClient implements HttpClient, DataSegmentChangeHandler + { + private final List segmentsSentToServer = new ArrayList<>(); + + @Override + public ListenableFuture go( + Request request, + HttpResponseHandler httpResponseHandler + ) + { + throw new UnsupportedOperationException("Not Implemented."); + } + + @Override + public ListenableFuture go( + Request request, + HttpResponseHandler httpResponseHandler, + Duration duration + ) + { + HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + httpResponse.setContent(ChannelBuffers.buffer(0)); + httpResponseHandler.handleResponse(httpResponse, null); + try { + List changeRequests = ServerTestHelper.MAPPER.readValue( + request.getContent().array(), new TypeReference>() + { + } + ); + + List statuses = new ArrayList<>(changeRequests.size()); + for (DataSegmentChangeRequest cr : changeRequests) { + cr.go(this, null); + statuses.add(new SegmentLoadDropHandler.DataSegmentChangeRequestAndStatus( + cr, + SegmentLoadDropHandler.Status.SUCCESS + )); + } + return (ListenableFuture) Futures.immediateFuture( + new ByteArrayInputStream( + ServerTestHelper.MAPPER + .writerWithType(HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF) + .writeValueAsBytes(statuses) + ) + ); + } + catch (Exception ex) { + throw new RE(ex, "Unexpected exception."); + } + } + + @Override + public void addSegment(DataSegment segment, DataSegmentChangeCallback callback) + { + segmentsSentToServer.add(segment); + } + + @Override + public void removeSegment(DataSegment segment, DataSegmentChangeCallback callback) + { + segmentsSentToServer.add(segment); + } + } + + /** + * Represents an action that can be performed on a segment by calling {@link #invoke()}. + */ + private static class QueueAction + { + final DataSegment segment; + final Consumer action; + + static QueueAction of(DataSegment segment, Consumer action) + { + return new QueueAction(segment, action); + } + + QueueAction(DataSegment segment, Consumer action) + { + this.segment = segment; + this.action = action; + } + + void invoke() + { + action.accept(segment); + } + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadQueuePeonTest.java similarity index 96% rename from server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java rename to server/src/test/java/org/apache/druid/server/coordinator/loading/LoadQueuePeonTest.java index 6b0f54d7803..aba1c9b6be6 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadQueuePeonTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.loading; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; @@ -38,6 +38,7 @@ import org.apache.druid.server.coordination.DataSegmentChangeHandler; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; +import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -212,11 +213,12 @@ public class LoadQueuePeonTest extends CuratorTestBase for (final DataSegment segment : segmentToLoad) { loadQueuePeon.loadSegment( segment, + SegmentAction.LOAD, success -> segmentLoadedSignals.get(segment.getId()).countDown() ); } - Assert.assertEquals(6000, loadQueuePeon.getLoadQueueSize()); + Assert.assertEquals(6000, loadQueuePeon.getSizeOfSegmentsToLoad()); Assert.assertEquals(5, loadQueuePeon.getSegmentsToLoad().size()); Assert.assertEquals(5, loadQueuePeon.getSegmentsToDrop().size()); Assert.assertEquals(0, loadQueuePeon.getTimedOutSegments().size()); @@ -290,12 +292,13 @@ public class LoadQueuePeonTest extends CuratorTestBase loadQueuePeon.loadSegment( segment, + SegmentAction.LOAD, success -> segmentLoadedSignal.countDown() ); Assert.assertTrue(timing.forWaiting().awaitLatch(segmentLoadedSignal)); Assert.assertEquals(0, loadQueuePeon.getSegmentsToLoad().size()); - Assert.assertEquals(0L, loadQueuePeon.getLoadQueueSize()); + Assert.assertEquals(0L, loadQueuePeon.getSizeOfSegmentsToLoad()); Assert.assertEquals(0, loadQueuePeon.getTimedOutSegments().size()); } @@ -351,6 +354,7 @@ public class LoadQueuePeonTest extends CuratorTestBase loadQueuePeon.loadSegment( segment, + SegmentAction.LOAD, success -> { segmentLoadedSignal.countDown(); delayedSegmentLoadedSignal.countDown(); @@ -371,7 +375,7 @@ public class LoadQueuePeonTest extends CuratorTestBase // simulate incompletion of load request since request has timed out Assert.assertTrue(timing.forWaiting().awaitLatch(segmentLoadedSignal)); Assert.assertEquals(1, loadQueuePeon.getSegmentsToLoad().size()); - Assert.assertEquals(1200L, loadQueuePeon.getLoadQueueSize()); + Assert.assertEquals(1200L, loadQueuePeon.getSizeOfSegmentsToLoad()); Assert.assertEquals(1, loadQueuePeon.getTimedOutSegments().size()); // simulate completion of load request by historical after time out on coordinator @@ -379,7 +383,7 @@ public class LoadQueuePeonTest extends CuratorTestBase Assert.assertTrue(timing.forWaiting().awaitLatch(delayedSegmentLoadedSignal)); Assert.assertTrue(timing.forWaiting().awaitLatch(loadRequestRemoveSignal)); Assert.assertEquals(0, loadQueuePeon.getSegmentsToLoad().size()); - Assert.assertEquals(0L, loadQueuePeon.getLoadQueueSize()); + Assert.assertEquals(0L, loadQueuePeon.getSizeOfSegmentsToLoad()); Assert.assertEquals(0, loadQueuePeon.getTimedOutSegments().size()); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadQueuePeonTester.java similarity index 85% rename from server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java rename to server/src/test/java/org/apache/druid/server/coordinator/loading/LoadQueuePeonTester.java index a7b92eaf8d0..e1a22aaa078 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/LoadQueuePeonTester.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/LoadQueuePeonTester.java @@ -17,17 +17,19 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.server.coordinator.loading; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; import org.apache.druid.timeline.DataSegment; import org.joda.time.Duration; +import javax.annotation.Nullable; import java.util.concurrent.ConcurrentSkipListSet; public class LoadQueuePeonTester extends CuratorLoadQueuePeon { - private final ConcurrentSkipListSet segmentsToLoad = new ConcurrentSkipListSet(); + private final ConcurrentSkipListSet segmentsToLoad = new ConcurrentSkipListSet<>(); public LoadQueuePeonTester() { @@ -46,10 +48,7 @@ public class LoadQueuePeonTester extends CuratorLoadQueuePeon } @Override - public void loadSegment( - DataSegment segment, - LoadPeonCallback callback - ) + public void loadSegment(DataSegment segment, SegmentAction action, @Nullable LoadPeonCallback callback) { segmentsToLoad.add(segment); } @@ -59,10 +58,4 @@ public class LoadQueuePeonTester extends CuratorLoadQueuePeon { return segmentsToLoad; } - - @Override - public int getNumberOfSegmentsInQueue() - { - return segmentsToLoad.size(); - } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java index 08e0616ee41..acf815089bb 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java @@ -23,14 +23,15 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; -import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCluster; -import org.apache.druid.server.coordinator.DruidClusterBuilder; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.LoadQueuePeonTester; -import org.apache.druid.server.coordinator.SegmentReplicantLookup; import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.balancer.RandomBalancerStrategy; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; +import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.junit.Assert; @@ -54,12 +55,18 @@ public class BroadcastDistributionRuleTest private ServerHolder activeServer; private ServerHolder decommissioningServer1; private ServerHolder decommissioningServer2; + private SegmentLoadQueueManager loadQueueManager; + + private static final String DS_SMALL = "small_source"; + private static final String TIER_1 = "tier1"; + private static final String TIER_2 = "tier2"; @Before public void setUp() { + loadQueueManager = new SegmentLoadQueueManager(null, null, null); smallSegment = new DataSegment( - "small_source", + DS_SMALL, Intervals.of("0/1000"), DateTimes.nowUtc().toString(), new HashMap<>(), @@ -109,7 +116,7 @@ public class BroadcastDistributionRuleTest null, 1000, ServerType.HISTORICAL, - "hot", + TIER_1, 0 ).addDataSegment(smallSegment) .toImmutableDruidServer(), @@ -124,7 +131,7 @@ public class BroadcastDistributionRuleTest null, 1000, ServerType.HISTORICAL, - "hot", + TIER_1, 0 ).addDataSegment(largeSegments.get(0)) .toImmutableDruidServer(), @@ -139,7 +146,7 @@ public class BroadcastDistributionRuleTest null, 1000, ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, + TIER_2, 0 ).addDataSegment(largeSegments.get(1)) .toImmutableDruidServer(), @@ -154,7 +161,7 @@ public class BroadcastDistributionRuleTest null, 100, ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, + TIER_2, 0 ).addDataSegment(largeSegments.get(2)) .toImmutableDruidServer(), @@ -170,7 +177,7 @@ public class BroadcastDistributionRuleTest null, 1000, ServerType.HISTORICAL, - "hot", + TIER_1, 0 ).addDataSegment(largeSegments2.get(0)) .toImmutableDruidServer(), @@ -185,7 +192,7 @@ public class BroadcastDistributionRuleTest null, 100, ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, + TIER_2, 0 ).addDataSegment(largeSegments2.get(1)) .toImmutableDruidServer(), @@ -200,7 +207,7 @@ public class BroadcastDistributionRuleTest null, 100, ServerType.HISTORICAL, - "tier1", + TIER_1, 0 ).addDataSegment(largeSegments.get(0)) .toImmutableDruidServer(), @@ -214,7 +221,7 @@ public class BroadcastDistributionRuleTest null, 100, ServerType.HISTORICAL, - "tier1", + TIER_1, 0 ).addDataSegment(smallSegment) .toImmutableDruidServer(), @@ -229,7 +236,7 @@ public class BroadcastDistributionRuleTest null, 100, ServerType.HISTORICAL, - "tier1", + TIER_1, 0 ).addDataSegment(largeSegments.get(1)) .toImmutableDruidServer(), @@ -237,26 +244,26 @@ public class BroadcastDistributionRuleTest true ); - druidCluster = DruidClusterBuilder - .newBuilder() + druidCluster = DruidCluster + .builder() .addTier( - "hot", + TIER_1, holdersOfLargeSegments.get(0), holderOfSmallSegment, holdersOfLargeSegments2.get(0) ) .addTier( - DruidServer.DEFAULT_TIER, + TIER_2, holdersOfLargeSegments.get(1), holdersOfLargeSegments.get(2), holdersOfLargeSegments2.get(1) ) .build(); - secondCluster = DruidClusterBuilder - .newBuilder() + secondCluster = DruidCluster + .builder() .addTier( - "tier1", + TIER_1, activeServer, decommissioningServer1, decommissioningServer2 @@ -270,8 +277,9 @@ public class BroadcastDistributionRuleTest final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(); - CoordinatorStats stats = rule.run( - null, + CoordinatorRunStats stats = runRuleAndGetStats( + rule, + smallSegment, makeCoordinartorRuntimeParams( druidCluster, smallSegment, @@ -280,36 +288,36 @@ public class BroadcastDistributionRuleTest largeSegments.get(2), largeSegments2.get(0), largeSegments2.get(1) - ), - smallSegment + ) ); - Assert.assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); - Assert.assertFalse(stats.hasPerTierStats()); + Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, DS_SMALL)); + Assert.assertEquals(3L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_2, DS_SMALL)); Assert.assertTrue( - holdersOfLargeSegments.stream() - .allMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) + holdersOfLargeSegments.stream().allMatch( + holder -> holder.isLoadingSegment(smallSegment) + ) ); - Assert.assertTrue( - holdersOfLargeSegments2.stream() - .allMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) + holdersOfLargeSegments2.stream().allMatch( + holder -> holder.isLoadingSegment(smallSegment) + ) ); - Assert.assertTrue(holderOfSmallSegment.isServingSegment(smallSegment)); } - private static DruidCoordinatorRuntimeParams makeCoordinartorRuntimeParams( + private DruidCoordinatorRuntimeParams makeCoordinartorRuntimeParams( DruidCluster druidCluster, DataSegment... usedSegments ) { - return CoordinatorRuntimeParamsTestHelpers - .newBuilder() + return DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc()) .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false)) .withUsedSegmentsInTest(usedSegments) + .withBalancerStrategy(new RandomBalancerStrategy()) + .withSegmentAssignerUsing(loadQueueManager) .build(); } @@ -332,20 +340,18 @@ public class BroadcastDistributionRuleTest final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(); - CoordinatorStats stats = rule.run( - null, + CoordinatorRunStats stats = runRuleAndGetStats( + rule, + smallSegment, makeCoordinartorRuntimeParams( secondCluster, smallSegment, largeSegments.get(0), largeSegments.get(1) - ), - smallSegment + ) ); - Assert.assertEquals(1L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); - Assert.assertFalse(stats.hasPerTierStats()); - + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, DS_SMALL)); Assert.assertEquals(1, activeServer.getPeon().getSegmentsToLoad().size()); Assert.assertEquals(1, decommissioningServer1.getPeon().getSegmentsToDrop().size()); Assert.assertEquals(0, decommissioningServer2.getPeon().getSegmentsToLoad().size()); @@ -354,11 +360,11 @@ public class BroadcastDistributionRuleTest @Test public void testBroadcastToMultipleDataSources() { - final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule( - ); + final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(); - CoordinatorStats stats = rule.run( - null, + CoordinatorRunStats stats = runRuleAndGetStats( + rule, + smallSegment, makeCoordinartorRuntimeParams( druidCluster, smallSegment, @@ -367,24 +373,23 @@ public class BroadcastDistributionRuleTest largeSegments.get(2), largeSegments2.get(0), largeSegments2.get(1) - ), - smallSegment + ) ); - Assert.assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); - Assert.assertFalse(stats.hasPerTierStats()); + Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, DS_SMALL)); + Assert.assertEquals(3L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_2, DS_SMALL)); Assert.assertTrue( - holdersOfLargeSegments.stream() - .allMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) + holdersOfLargeSegments.stream().allMatch( + holder -> holder.isLoadingSegment(smallSegment) + ) ); - Assert.assertTrue( - holdersOfLargeSegments2.stream() - .allMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) + holdersOfLargeSegments2.stream().allMatch( + holder -> holder.isLoadingSegment(smallSegment) + ) ); - - Assert.assertFalse(holderOfSmallSegment.getPeon().getSegmentsToLoad().contains(smallSegment)); + Assert.assertFalse(holderOfSmallSegment.isLoadingSegment(smallSegment)); } @Test @@ -392,8 +397,9 @@ public class BroadcastDistributionRuleTest { final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(); - CoordinatorStats stats = rule.run( - null, + CoordinatorRunStats stats = runRuleAndGetStats( + rule, + smallSegment, makeCoordinartorRuntimeParams( druidCluster, smallSegment, @@ -402,18 +408,27 @@ public class BroadcastDistributionRuleTest largeSegments.get(2), largeSegments2.get(0), largeSegments2.get(1) - ), - smallSegment + ) ); - Assert.assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); - Assert.assertFalse(stats.hasPerTierStats()); + Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, DS_SMALL)); + Assert.assertEquals(3L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_2, DS_SMALL)); Assert.assertTrue( - druidCluster - .getAllServers() - .stream() - .allMatch(holder -> holder.isLoadingSegment(smallSegment) || holder.isServingSegment(smallSegment)) + druidCluster.getAllServers().stream().allMatch( + holder -> holder.isLoadingSegment(smallSegment) || holder.isServingSegment(smallSegment) + ) ); } + + private CoordinatorRunStats runRuleAndGetStats( + Rule rule, + DataSegment segment, + DruidCoordinatorRuntimeParams params + ) + { + StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); + rule.run(segment, segmentAssigner); + return segmentAssigner.getStats(); + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java index 642823d186c..85aeeb77db1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java @@ -19,90 +19,71 @@ package org.apache.druid.server.coordinator.rules; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.core.LoggingEmitter; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.BalancerStrategy; -import org.apache.druid.server.coordinator.CachingCostBalancerStrategy; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; -import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; -import org.apache.druid.server.coordinator.CoordinatorStats; -import org.apache.druid.server.coordinator.CostBalancerStrategyFactory; +import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCluster; -import org.apache.druid.server.coordinator.DruidClusterBuilder; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.LoadQueuePeon; -import org.apache.druid.server.coordinator.LoadQueuePeonTester; -import org.apache.druid.server.coordinator.ReplicationThrottler; -import org.apache.druid.server.coordinator.RoundRobinServerSelector; -import org.apache.druid.server.coordinator.SegmentReplicantLookup; import org.apache.druid.server.coordinator.ServerHolder; -import org.apache.druid.server.coordinator.cost.ClusterCostCache; +import org.apache.druid.server.coordinator.balancer.BalancerStrategy; +import org.apache.druid.server.coordinator.balancer.CachingCostBalancerStrategy; +import org.apache.druid.server.coordinator.balancer.ClusterCostCache; +import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory; +import org.apache.druid.server.coordinator.loading.LoadQueuePeon; +import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; +import org.apache.druid.server.coordinator.loading.SegmentAction; +import org.apache.druid.server.coordinator.loading.SegmentHolder; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; +import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; -import org.joda.time.DateTime; -import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; /** + * */ @RunWith(Parameterized.class) public class LoadRuleTest { - private static final Logger log = new Logger(LoadRuleTest.class); - private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); - - private static final ServiceEmitter EMITTER = new ServiceEmitter( - "service", - "host", - new LoggingEmitter( - log, - LoggingEmitter.Level.ERROR, - JSON_MAPPER - ) - ); - - private ReplicationThrottler throttler; + private static final String DS_WIKI = "wiki"; private ListeningExecutorService exec; private BalancerStrategy balancerStrategy; private CachingCostBalancerStrategy cachingCostBalancerStrategy; + private SegmentLoadQueueManager loadQueueManager; private final boolean useRoundRobinAssignment; private BalancerStrategy mockBalancerStrategy; + private final AtomicInteger serverId = new AtomicInteger(); + @Parameterized.Parameters(name = "useRoundRobin = {0}") public static List getTestParams() { @@ -117,84 +98,68 @@ public class LoadRuleTest @Before public void setUp() { - EmittingLogger.registerEmitter(EMITTER); - EMITTER.start(); - throttler = EasyMock.createMock(ReplicationThrottler.class); - exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "LoadRuleTest-%d")); balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); cachingCostBalancerStrategy = new CachingCostBalancerStrategy(ClusterCostCache.builder().build(), exec); mockBalancerStrategy = EasyMock.createMock(BalancerStrategy.class); + loadQueueManager = new SegmentLoadQueueManager(null, null, null); } @After - public void tearDown() throws Exception + public void tearDown() { exec.shutdown(); - EMITTER.close(); } @Test public void testLoad() { - EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes(); - final LoadQueuePeon mockPeon = createEmptyPeon(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - LoadRule rule = createLoadRule(ImmutableMap.of( - "hot", 1, - DruidServer.DEFAULT_TIER, 2 - )); - - final DataSegment segment = createDataSegment("foo"); - - throttler.registerReplicantCreation(DruidServer.DEFAULT_TIER, segment.getId(), "hostNorm"); - EasyMock.expectLastCall().once(); - if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) - .times(3); + .times(2); } + EasyMock.replay(mockPeon, mockBalancerStrategy); - EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "hot", - new ServerHolder( - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1) - .toImmutableDruidServer(), - mockPeon - ) - ) - .addTier( - DruidServer.DEFAULT_TIER, - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ) + DruidCluster druidCluster = DruidCluster + .builder() + .addTier(Tier.T1, createServerHolder(Tier.T1, mockPeon, false)) + .addTier(Tier.T2, createServerHolder(Tier.T2, mockPeon, false)) .build(); - CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); + final DataSegment segment = createDataSegment(DS_WIKI); + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 2)); + CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI)); - EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); + EasyMock.verify(mockPeon, mockBalancerStrategy); + } + + private CoordinatorRunStats runRuleAndGetStats( + LoadRule rule, + DataSegment segment, + DruidCluster cluster + ) + { + return runRuleAndGetStats(rule, segment, makeCoordinatorRuntimeParams(cluster, segment)); + } + + private CoordinatorRunStats runRuleAndGetStats( + LoadRule rule, + DataSegment segment, + DruidCoordinatorRuntimeParams params + ) + { + final StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); + rule.run(segment, segmentAssigner); + return segmentAssigner.getStats(); } private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParams( @@ -202,312 +167,199 @@ public class LoadRuleTest DataSegment... usedSegments ) { - return CoordinatorRuntimeParamsTestHelpers - .newBuilder() + return DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc()) .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false)) - .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withUsedSegmentsInTest(usedSegments) .withDynamicConfigs( - CoordinatorDynamicConfig - .builder() - .withUseRoundRobinSegmentAssignment(useRoundRobinAssignment) - .build() + CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withUseRoundRobinSegmentAssignment(useRoundRobinAssignment) + .build() ) - .withRoundRobinServerSelector(useRoundRobinAssignment ? new RoundRobinServerSelector(druidCluster) : null) - .withBalancerStrategy(mockBalancerStrategy) - .withUsedSegmentsInTest(usedSegments) - .build(); - } - - private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParamsWithLoadReplicationOnTimeout( - DruidCluster druidCluster, - DataSegment... usedSegments - ) - { - return CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, true)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withUsedSegmentsInTest(usedSegments) + .withSegmentAssignerUsing(loadQueueManager) .build(); } @Test public void testLoadPrimaryAssignDoesNotOverAssign() { - EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes(); - final LoadQueuePeon mockPeon = createEmptyPeon(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - LoadRule rule = createLoadRule(ImmutableMap.of( - "hot", 1 - )); + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - final DataSegment segment = createDataSegment("foo"); + final DataSegment segment = createDataSegment(DS_WIKI); - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) .anyTimes(); - EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon, mockBalancerStrategy); - ImmutableDruidServer server1 = - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); - ImmutableDruidServer server2 = - new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("hot", new ServerHolder(server1, mockPeon), new ServerHolder(server2, mockPeon)) + ImmutableDruidServer server1 = createServer(Tier.T1).toImmutableDruidServer(); + ImmutableDruidServer server2 = createServer(Tier.T1).toImmutableDruidServer(); + DruidCluster druidCluster = DruidCluster + .builder() + .addTier(Tier.T1, new ServerHolder(server1, mockPeon), new ServerHolder(server2, mockPeon)) .build(); - CoordinatorStats stats = rule.run( - null, - makeCoordinatorRuntimeParams(druidCluster, segment), - segment - ); - - - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); + CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, segment.getDataSource())); // ensure multiple runs don't assign primary segment again if at replication count - final LoadQueuePeon loadingPeon = createLoadingPeon(ImmutableList.of(segment), false); + final LoadQueuePeon loadingPeon = createLoadingPeon(segment, false); EasyMock.replay(loadingPeon); - DruidCluster afterLoad = DruidClusterBuilder - .newBuilder() - .addTier("hot", new ServerHolder(server1, loadingPeon), new ServerHolder(server2, mockPeon)) + DruidCluster afterLoad = DruidCluster + .builder() + .addTier(Tier.T1, new ServerHolder(server1, loadingPeon), new ServerHolder(server2, mockPeon)) .build(); - CoordinatorStats statsAfterLoadPrimary = rule.run( - null, - makeCoordinatorRuntimeParams(afterLoad, segment), - segment - ); + CoordinatorRunStats statsAfterLoadPrimary = runRuleAndGetStats(rule, segment, afterLoad); - Assert.assertEquals(0, statsAfterLoadPrimary.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); + Assert.assertFalse(statsAfterLoadPrimary.hasStat(Stats.Segments.ASSIGNED)); - EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); + EasyMock.verify(mockPeon, mockBalancerStrategy); } @Test + @Ignore("Enable this test when timeout behaviour is fixed") public void testOverAssignForTimedOutSegments() { - EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes(); - final LoadQueuePeon emptyPeon = createEmptyPeon(); - emptyPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); + emptyPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - LoadRule rule = createLoadRule(ImmutableMap.of( - "hot", 1 - )); + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - final DataSegment segment = createDataSegment("foo"); + final DataSegment segment = createDataSegment(DS_WIKI); - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) .anyTimes(); - EasyMock.replay(throttler, emptyPeon, mockBalancerStrategy); + EasyMock.replay(emptyPeon, mockBalancerStrategy); - ImmutableDruidServer server1 = - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); - ImmutableDruidServer server2 = - new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("hot", new ServerHolder(server1, emptyPeon), new ServerHolder(server2, emptyPeon)) + ImmutableDruidServer server1 = createServer(Tier.T1).toImmutableDruidServer(); + ImmutableDruidServer server2 = createServer(Tier.T1).toImmutableDruidServer(); + DruidCluster druidCluster = DruidCluster + .builder() + .addTier(Tier.T1, new ServerHolder(server1, emptyPeon), new ServerHolder(server2, emptyPeon)) .build(); - CoordinatorStats stats = rule.run( - null, - makeCoordinatorRuntimeParamsWithLoadReplicationOnTimeout(druidCluster, segment), - segment + CoordinatorRunStats stats = runRuleAndGetStats( + rule, + segment, + makeCoordinatorRuntimeParams(druidCluster, segment) ); // Ensure that the segment is assigned to one of the historicals - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, segment.getDataSource())); // Ensure that the primary segment is assigned again in case the peon timed out on loading the segment - final LoadQueuePeon slowLoadingPeon = createLoadingPeon(ImmutableList.of(segment), true); + final LoadQueuePeon slowLoadingPeon = createLoadingPeon(segment, true); EasyMock.replay(slowLoadingPeon); - DruidCluster withLoadTimeout = DruidClusterBuilder - .newBuilder() - .addTier("hot", new ServerHolder(server1, slowLoadingPeon), new ServerHolder(server2, emptyPeon)) + DruidCluster withLoadTimeout = DruidCluster + .builder() + .addTier(Tier.T1, new ServerHolder(server1, slowLoadingPeon), new ServerHolder(server2, emptyPeon)) .build(); - CoordinatorStats statsAfterLoadPrimary = rule.run( - null, - makeCoordinatorRuntimeParamsWithLoadReplicationOnTimeout(withLoadTimeout, segment), - segment + CoordinatorRunStats statsAfterLoadPrimary = runRuleAndGetStats( + rule, + segment, + makeCoordinatorRuntimeParams(withLoadTimeout, segment) ); - Assert.assertEquals(1L, statsAfterLoadPrimary.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); + Assert.assertEquals(1L, statsAfterLoadPrimary.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); - EasyMock.verify(throttler, emptyPeon, mockBalancerStrategy); + EasyMock.verify(emptyPeon, mockBalancerStrategy); } @Test public void testSkipReplicationForTimedOutSegments() { - EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes(); - final LoadQueuePeon emptyPeon = createEmptyPeon(); - emptyPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); + emptyPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - LoadRule rule = createLoadRule(ImmutableMap.of( - "hot", 1 - )); + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - final DataSegment segment = createDataSegment("foo"); + final DataSegment segment = createDataSegment(DS_WIKI); - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) .anyTimes(); - EasyMock.replay(throttler, emptyPeon, mockBalancerStrategy); + EasyMock.replay(emptyPeon, mockBalancerStrategy); - ImmutableDruidServer server1 = - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); - ImmutableDruidServer server2 = - new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("hot", new ServerHolder(server1, emptyPeon), new ServerHolder(server2, emptyPeon)) + ImmutableDruidServer server1 = createServer(Tier.T1).toImmutableDruidServer(); + ImmutableDruidServer server2 = createServer(Tier.T1).toImmutableDruidServer(); + DruidCluster druidCluster = DruidCluster + .builder() + .addTier(Tier.T1, new ServerHolder(server1, emptyPeon), new ServerHolder(server2, emptyPeon)) .build(); - CoordinatorStats stats = rule.run( - null, - makeCoordinatorRuntimeParams(druidCluster, segment), - segment - ); + CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); // Ensure that the segment is assigned to one of the historicals - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, segment.getDataSource())); // Add the segment to the timed out list to simulate peon timeout on loading the segment - final LoadQueuePeon slowLoadingPeon = createLoadingPeon(ImmutableList.of(segment), true); + final LoadQueuePeon slowLoadingPeon = createLoadingPeon(segment, true); EasyMock.replay(slowLoadingPeon); - DruidCluster withLoadTimeout = DruidClusterBuilder - .newBuilder() - .addTier("hot", new ServerHolder(server1, slowLoadingPeon), new ServerHolder(server2, emptyPeon)) + DruidCluster withLoadTimeout = DruidCluster + .builder() + .addTier(Tier.T1, new ServerHolder(server1, slowLoadingPeon), new ServerHolder(server2, emptyPeon)) .build(); // Default behavior is to not replicate the timed out segments on other servers - CoordinatorStats statsAfterLoadPrimary = rule.run( - null, - makeCoordinatorRuntimeParams(withLoadTimeout, segment), - segment - ); + CoordinatorRunStats statsAfterLoadPrimary = runRuleAndGetStats(rule, segment, withLoadTimeout); - Assert.assertEquals(0L, statsAfterLoadPrimary.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); + Assert.assertFalse(statsAfterLoadPrimary.hasStat(Stats.Segments.ASSIGNED)); - EasyMock.verify(throttler, emptyPeon, mockBalancerStrategy); + EasyMock.verify(emptyPeon, mockBalancerStrategy); } @Test public void testLoadUsedSegmentsForAllSegmentGranularityAndCachingCostBalancerStrategy() { - EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(false).anyTimes(); + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 1)); + final List segments = + CreateDataSegments.ofDatasource(DS_WIKI) + .forIntervals(1, Granularities.ALL) + .withNumPartitions(2) + .eachOfSizeInMb(100); - DataSegment segment0 = createDataSegmentWithIntervalAndPartition(createDataSegment("foo"), - JodaUtils.MIN_INSTANT, - JodaUtils.MAX_INSTANT, - 0); - DataSegment segment1 = createDataSegmentWithIntervalAndPartition(createDataSegment("foo"), - JodaUtils.MIN_INSTANT, - JodaUtils.MAX_INSTANT, - 1); + final LoadQueuePeon loadingPeon = createLoadingPeon(segments.get(0), true); - final LoadQueuePeon loadingPeon = createLoadingPeon(ImmutableList.of(segment0), true); - - loadingPeon.loadSegment(EasyMock.anyObject(), EasyMock.isNull()); + loadingPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().once(); - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(cachingCostBalancerStrategy) .anyTimes(); - EasyMock.replay(throttler, loadingPeon, mockBalancerStrategy); + EasyMock.replay(loadingPeon, mockBalancerStrategy); - ImmutableDruidServer server = - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "tier1", 1).toImmutableDruidServer(); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("tier1", new ServerHolder(server, loadingPeon)) + DruidCluster druidCluster = DruidCluster + .builder() + .addTier(Tier.T1, createServerHolder(Tier.T1, loadingPeon, false)) .build(); - final CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParamsWithLoadReplicationOnTimeout(druidCluster, segment0, segment1), segment1); + CoordinatorRunStats stats = runRuleAndGetStats( + rule, + segments.get(1), + makeCoordinatorRuntimeParams(druidCluster, segments.toArray(new DataSegment[0])) + ); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); - - EasyMock.verify(throttler, loadingPeon, mockBalancerStrategy); - } - - @Test - public void testLoadPriority() - { - EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(false).anyTimes(); - - final LoadQueuePeon mockPeon1 = createEmptyPeon(); - final LoadQueuePeon mockPeon2 = createEmptyPeon(); - - mockPeon2.loadSegment(EasyMock.anyObject(), EasyMock.isNull()); - EasyMock.expectLastCall().once(); - - if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .times(2); - } - - EasyMock.replay(throttler, mockPeon1, mockPeon2, mockBalancerStrategy); - - final LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 10, "tier2", 10)); - - final DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "tier1", - new ServerHolder( - new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "tier1", 0) - .toImmutableDruidServer(), - mockPeon1 - ) - ) - .addTier( - "tier2", - new ServerHolder( - new DruidServer("server2", "host2", null, 1000, ServerType.HISTORICAL, "tier2", 1) - .toImmutableDruidServer(), - mockPeon2 - ), - new ServerHolder( - new DruidServer("server3", "host3", null, 1000, ServerType.HISTORICAL, "tier2", 1) - .toImmutableDruidServer(), - mockPeon2 - ) - ) - .build(); - - final DataSegment segment = createDataSegment("foo"); - - final CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); - - Assert.assertEquals(0L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); - - EasyMock.verify(throttler, mockPeon1, mockPeon2, mockBalancerStrategy); + EasyMock.verify(loadingPeon, mockBalancerStrategy); } @Test @@ -516,109 +368,66 @@ public class LoadRuleTest final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.pickServersToDropSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) .times(4); - EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon, mockBalancerStrategy); - LoadRule rule = createLoadRule(ImmutableMap.of( - "hot", 0, - DruidServer.DEFAULT_TIER, 0 - )); + final DataSegment segment = createDataSegment(DS_WIKI); - final DataSegment segment = createDataSegment("foo"); - - DruidServer server1 = new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0); + DruidServer server1 = createServer(Tier.T1); server1.addDataSegment(segment); - DruidServer server2 = new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ); + DruidServer server2 = createServer(Tier.T2); server2.addDataSegment(segment); - DruidServer server3 = new DruidServer( - "serverNormNotServing", - "hostNorm", - null, - 10, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ); - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + DruidServer server3 = createServer(Tier.T2); + + DruidCluster druidCluster = DruidCluster + .builder() + .addTier(Tier.T1, new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) .addTier( - DruidServer.DEFAULT_TIER, + Tier.T2, new ServerHolder(server2.toImmutableDruidServer(), mockPeon), new ServerHolder(server3.toImmutableDruidServer(), mockPeon) ) .build(); - CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 0, Tier.T2, 0)); + CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); - Assert.assertEquals(1L, stats.getTieredStat("droppedCount", DruidServer.DEFAULT_TIER)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T2, DS_WIKI)); - EasyMock.verify(throttler, mockPeon); + EasyMock.verify(mockPeon); } @Test public void testLoadWithNonExistentTier() { final LoadQueuePeon mockPeon = createEmptyPeon(); - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) .times(1); } - EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon, mockBalancerStrategy); - LoadRule rule = createLoadRule(ImmutableMap.of("nonExistentTier", 1, "hot", 1)); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier( - "hot", - new ServerHolder( - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), - mockPeon - ) - ) + DruidCluster druidCluster = DruidCluster + .builder() + .addTier(Tier.T1, createServerHolder(Tier.T1, mockPeon, false)) .build(); - final DataSegment segment = createDataSegment("foo"); + final DataSegment segment = createDataSegment(DS_WIKI); + LoadRule rule = loadForever(ImmutableMap.of("nonExistentTier", 1, Tier.T1, 1)); - final CoordinatorDynamicConfig dynamicConfig = - CoordinatorDynamicConfig.builder() - .withUseRoundRobinSegmentAssignment(useRoundRobinAssignment) - .build(); - CoordinatorStats stats = rule.run( - null, - CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster(), false)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withDynamicConfigs(dynamicConfig) - .withUsedSegmentsInTest(segment) - .build(), - segment - ); + CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); - EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); + EasyMock.verify(mockPeon, mockBalancerStrategy); } @Test @@ -627,62 +436,55 @@ public class LoadRuleTest final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .times(2); - EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon, mockBalancerStrategy); - LoadRule rule = createLoadRule(ImmutableMap.of("nonExistentTier", 1, "hot", 1)); + final DataSegment segment = createDataSegment(DS_WIKI); - final DataSegment segment = createDataSegment("foo"); - - DruidServer server1 = new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0); - DruidServer server2 = new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0); + DruidServer server1 = createServer(Tier.T1); + DruidServer server2 = createServer(Tier.T1); server1.addDataSegment(segment); server2.addDataSegment(segment); - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() + DruidCluster druidCluster = DruidCluster + .builder() .addTier( - "hot", + Tier.T1, new ServerHolder(server1.toImmutableDruidServer(), mockPeon), new ServerHolder(server2.toImmutableDruidServer(), mockPeon) ) .build(); - CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); + LoadRule rule = loadForever(ImmutableMap.of("nonExistentTier", 1, Tier.T1, 1)); + CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, DS_WIKI)); - EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); + EasyMock.verify(mockPeon, mockBalancerStrategy); } @Test public void testMaxLoadingQueueSize() { - final int maxSegmentsInLoadQueue = 2; if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) .times(2); } - - EasyMock.replay(throttler, mockBalancerStrategy); + EasyMock.replay(mockBalancerStrategy); final LoadQueuePeonTester peon = new LoadQueuePeonTester(); - LoadRule rule = createLoadRule(ImmutableMap.of("hot", 1)); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() + final int maxSegmentsInQueue = 2; + DruidCluster druidCluster = DruidCluster + .builder() .addTier( - "hot", + Tier.T1, new ServerHolder( - new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) - .toImmutableDruidServer(), + createServer(Tier.T1).toImmutableDruidServer(), peon, false, - maxSegmentsInLoadQueue + maxSegmentsInQueue, + 10 ) ) .build(); @@ -691,29 +493,31 @@ public class LoadRuleTest DataSegment dataSegment2 = createDataSegment("ds2"); DataSegment dataSegment3 = createDataSegment("ds3"); - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() + DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + .newBuilder(DateTimes.nowUtc()) .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false)) - .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withUsedSegmentsInTest(dataSegment1, dataSegment2, dataSegment3) .withDynamicConfigs( CoordinatorDynamicConfig.builder() - .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInLoadQueue) + .withSmartSegmentLoading(false) + .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue) .withUseRoundRobinSegmentAssignment(useRoundRobinAssignment) .build() - ).build(); + ) + .withSegmentAssignerUsing(loadQueueManager) + .build(); - CoordinatorStats stats1 = rule.run(null, params, dataSegment1); - CoordinatorStats stats2 = rule.run(null, params, dataSegment2); - CoordinatorStats stats3 = rule.run(null, params, dataSegment3); + final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); + CoordinatorRunStats stats1 = runRuleAndGetStats(rule, dataSegment1, params); + CoordinatorRunStats stats2 = runRuleAndGetStats(rule, dataSegment2, params); + CoordinatorRunStats stats3 = runRuleAndGetStats(rule, dataSegment3, params); - Assert.assertEquals(1L, stats1.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); - Assert.assertEquals(1L, stats2.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); - Assert.assertFalse(stats3.getTiers(LoadRule.ASSIGNED_COUNT).contains("hot")); + Assert.assertEquals(1L, stats1.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, dataSegment1.getDataSource())); + Assert.assertEquals(1L, stats2.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, dataSegment2.getDataSource())); + Assert.assertEquals(0L, stats3.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, dataSegment3.getDataSource())); - EasyMock.verify(throttler, mockBalancerStrategy); + EasyMock.verify(mockBalancerStrategy); } /** @@ -726,28 +530,25 @@ public class LoadRuleTest final LoadQueuePeon mockPeon1 = createEmptyPeon(); final LoadQueuePeon mockPeon2 = createOneCallPeonMock(); - LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 1, "tier2", 1)); - - final DataSegment segment = createDataSegment("foo"); - if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) .times(1); } EasyMock.replay(mockPeon1, mockPeon2, mockBalancerStrategy); - - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("tier1", createServerHolder("tier1", mockPeon1, true)) - .addTier("tier2", createServerHolder("tier2", mockPeon2, false)) + DruidCluster druidCluster = DruidCluster + .builder() + .addTier(Tier.T1, createServerHolder(Tier.T1, mockPeon1, true)) + .addTier(Tier.T2, createServerHolder(Tier.T2, mockPeon2, false)) .build(); - CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 1)); + DataSegment segment = createDataSegment(DS_WIKI); + CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI)); EasyMock.verify(mockPeon1, mockPeon2, mockBalancerStrategy); } @@ -758,48 +559,40 @@ public class LoadRuleTest @Test public void testLoadReplicaDuringDecommissioning() { - EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes(); - final LoadQueuePeon mockPeon1 = createEmptyPeon(); final LoadQueuePeon mockPeon2 = createOneCallPeonMock(); final LoadQueuePeon mockPeon3 = createOneCallPeonMock(); final LoadQueuePeon mockPeon4 = createOneCallPeonMock(); + EasyMock.replay(mockPeon1, mockPeon2, mockPeon3, mockPeon4); - LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 2, "tier2", 2)); - - final DataSegment segment = createDataSegment("foo"); - - throttler.registerReplicantCreation(EasyMock.eq("tier2"), EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().times(2); - - ServerHolder holder1 = createServerHolder("tier1", mockPeon1, true); - ServerHolder holder2 = createServerHolder("tier1", mockPeon2, false); - ServerHolder holder3 = createServerHolder("tier2", mockPeon3, false); - ServerHolder holder4 = createServerHolder("tier2", mockPeon4, false); + ServerHolder holder1 = createServerHolder(Tier.T1, mockPeon1, true); + ServerHolder holder2 = createServerHolder(Tier.T1, mockPeon2, false); + ServerHolder holder3 = createServerHolder(Tier.T2, mockPeon3, false); + ServerHolder holder4 = createServerHolder(Tier.T2, mockPeon4, false); + final DataSegment segment = createDataSegment(DS_WIKI); if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(segment, ImmutableList.of(holder2))) - .andReturn(holder2); - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(segment, ImmutableList.of(holder4, holder3))) - .andReturn(holder3); - EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(segment, ImmutableList.of(holder4))) - .andReturn(holder4); + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(segment, ImmutableList.of(holder2))) + .andReturn(Collections.singletonList(holder2).iterator()); + EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(segment, ImmutableList.of(holder4, holder3))) + .andReturn(Arrays.asList(holder3, holder4).iterator()); } - EasyMock.replay(throttler, mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy); + EasyMock.replay(mockBalancerStrategy); - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() - .addTier("tier1", holder1, holder2) - .addTier("tier2", holder3, holder4) + DruidCluster druidCluster = DruidCluster + .builder() + .addTier(Tier.T1, holder1, holder2) + .addTier(Tier.T2, holder3, holder4) .build(); - CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 2, Tier.T2, 2)); + CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); - Assert.assertEquals(2L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); + Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI)); - EasyMock.verify(throttler, mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy); + EasyMock.verify(mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy); } /** @@ -812,37 +605,38 @@ public class LoadRuleTest final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().times(2); - EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.pickServersToDropSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) .times(4); - EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - - LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 0)); + EasyMock.replay(mockPeon, mockBalancerStrategy); final DataSegment segment1 = createDataSegment("foo1"); final DataSegment segment2 = createDataSegment("foo2"); - DruidServer server1 = createServer("tier1"); + DruidServer server1 = createServer(Tier.T1); server1.addDataSegment(segment1); - DruidServer server2 = createServer("tier1"); + DruidServer server2 = createServer(Tier.T1); server2.addDataSegment(segment2); - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() + DruidCluster druidCluster = DruidCluster + .builder() .addTier( - "tier1", + Tier.T1, new ServerHolder(server1.toImmutableDruidServer(), mockPeon, true), new ServerHolder(server2.toImmutableDruidServer(), mockPeon, false) ) .build(); DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, segment1, segment2); - CoordinatorStats stats = rule.run(null, params, segment1); - Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); - stats = rule.run(null, params, segment2); - Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); + final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 0)); - EasyMock.verify(throttler, mockPeon); + CoordinatorRunStats stats = runRuleAndGetStats(rule, segment1, params); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, segment1.getDataSource())); + + stats = runRuleAndGetStats(rule, segment2, params); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, segment2.getDataSource())); + + EasyMock.verify(mockPeon); } /** @@ -857,39 +651,41 @@ public class LoadRuleTest final LoadQueuePeon mockPeon1 = new LoadQueuePeonTester(); final LoadQueuePeon mockPeon2 = new LoadQueuePeonTester(); final LoadQueuePeon mockPeon3 = new LoadQueuePeonTester(); - EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject())) + EasyMock.expect(mockBalancerStrategy.pickServersToDropSegment(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) .times(4); - EasyMock.replay(throttler, mockBalancerStrategy); + EasyMock.replay(mockBalancerStrategy); - LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 2)); + final DataSegment segment1 = createDataSegment(DS_WIKI); - final DataSegment segment1 = createDataSegment("foo1"); - - DruidServer server1 = createServer("tier1"); + DruidServer server1 = createServer(Tier.T1); server1.addDataSegment(segment1); - DruidServer server2 = createServer("tier1"); + DruidServer server2 = createServer(Tier.T1); server2.addDataSegment(segment1); - DruidServer server3 = createServer("tier1"); + DruidServer server3 = createServer(Tier.T1); server3.addDataSegment(segment1); - DruidCluster druidCluster = DruidClusterBuilder - .newBuilder() + DruidCluster druidCluster = DruidCluster + .builder() .addTier( - "tier1", + Tier.T1, new ServerHolder(server1.toImmutableDruidServer(), mockPeon1, false), new ServerHolder(server2.toImmutableDruidServer(), mockPeon2, true), new ServerHolder(server3.toImmutableDruidServer(), mockPeon3, false) ) .build(); - CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment1), segment1); - Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 2)); + CoordinatorRunStats stats = runRuleAndGetStats( + rule, + segment1, + makeCoordinatorRuntimeParams(druidCluster, segment1) + ); + + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, DS_WIKI)); Assert.assertEquals(0, mockPeon1.getSegmentsToDrop().size()); Assert.assertEquals(1, mockPeon2.getSegmentsToDrop().size()); Assert.assertEquals(0, mockPeon3.getSegmentsToDrop().size()); - - EasyMock.verify(throttler); } private DataSegment createDataSegment(String dataSource) @@ -898,115 +694,63 @@ public class LoadRuleTest dataSource, Intervals.of("0/3000"), DateTimes.nowUtc().toString(), - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList(), NoneShardSpec.instance(), 0, 0 ); } - private DataSegment createDataSegmentWithIntervalAndPartition(DataSegment dataSegment, long startMillis, long endMillis, int partitionNum) + private static LoadRule loadForever(final Map tieredReplicants) { - return new DataSegment.Builder(dataSegment) - .interval(new Interval(startMillis, endMillis, dataSegment.getInterval().getChronology())) - .shardSpec(new LinearShardSpec(partitionNum)) - .build(); - } - - private static LoadRule createLoadRule(final Map tieredReplicants) - { - return new LoadRule() - { - @Override - public Map getTieredReplicants() - { - return tieredReplicants; - } - - @Override - public int getNumReplicants(String tier) - { - return tieredReplicants.get(tier); - } - - @Override - public String getType() - { - return "test"; - } - - @Override - public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) - { - return true; - } - - @Override - public boolean appliesTo(Interval interval, DateTime referenceTimestamp) - { - return true; - } - }; + return new ForeverLoadRule(tieredReplicants); } private static LoadQueuePeon createEmptyPeon() { final LoadQueuePeon mockPeon = EasyMock.createMock(LoadQueuePeon.class); - EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(new HashSet<>()).anyTimes(); - EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(new HashSet<>()).anyTimes(); - EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).anyTimes(); - EasyMock.expect(mockPeon.getNumberOfSegmentsInQueue()).andReturn(0).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsInQueue()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSizeOfSegmentsToLoad()).andReturn(0L).anyTimes(); return mockPeon; } - private static LoadQueuePeon createLoadingPeon(List segments, boolean slowLoading) + private static LoadQueuePeon createLoadingPeon(DataSegment segment, boolean slowLoading) { - final Set segs = ImmutableSet.copyOf(segments); - final long loadingSize = segs.stream().mapToLong(DataSegment::getSize).sum(); + final Set segs = Collections.singleton(segment); final LoadQueuePeon mockPeon = EasyMock.createMock(LoadQueuePeon.class); EasyMock.expect(mockPeon.getSegmentsToLoad()).andReturn(segs).anyTimes(); - EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(new HashSet<>()).anyTimes(); - EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(loadingSize).anyTimes(); - EasyMock.expect(mockPeon.getNumberOfSegmentsInQueue()).andReturn(segs.size()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsToDrop()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsInQueue()) + .andReturn(Collections.singleton(new SegmentHolder(segment, SegmentAction.LOAD, null))).anyTimes(); - if (slowLoading) { - EasyMock.expect(mockPeon.getTimedOutSegments()).andReturn(new HashSet<>(segments)).anyTimes(); - } else { - EasyMock.expect(mockPeon.getTimedOutSegments()).andReturn(new HashSet<>()).anyTimes(); - } + EasyMock.expect(mockPeon.getTimedOutSegments()) + .andReturn(slowLoading ? segs : Collections.emptySet()).anyTimes(); return mockPeon; } - private static final AtomicInteger SERVER_ID = new AtomicInteger(); - - private static DruidServer createServer(String tier) + private DruidServer createServer(String tier) { - int serverId = LoadRuleTest.SERVER_ID.incrementAndGet(); - return new DruidServer( - "server" + serverId, - "127.0.0.1:800" + serverId, - null, - 1000, - ServerType.HISTORICAL, - tier, - 0 - ); + final String serverName = "hist_" + tier + "_" + serverId.incrementAndGet(); + return new DruidServer(serverName, serverName, null, 10L << 30, ServerType.HISTORICAL, tier, 0); } private static LoadQueuePeon createOneCallPeonMock() { final LoadQueuePeon mockPeon2 = createEmptyPeon(); - mockPeon2.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); + mockPeon2.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().once(); return mockPeon2; } - private static ServerHolder createServerHolder(String tier, LoadQueuePeon mockPeon1, boolean isDecommissioning) + private ServerHolder createServerHolder(String tier, LoadQueuePeon mockPeon1, boolean isDecommissioning) { return new ServerHolder( createServer(tier).toImmutableDruidServer(), @@ -1014,4 +758,10 @@ public class LoadRuleTest isDecommissioning ); } + + private static class Tier + { + static final String T1 = "tier1"; + static final String T2 = "tier2"; + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java index f135280825e..89bff979a0c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java @@ -55,7 +55,7 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest } @Test - public void testFreshClusterGetsBalanced() + public void testNewClusterGetsBalanced() { final List historicals = new ArrayList<>(); for (int i = 0; i < 5; i++) { @@ -64,7 +64,6 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest CoordinatorSimulation sim = CoordinatorSimulation.builder() - .withDynamicConfig(createDynamicConfig(1000, 0, 100)) .withBalancer(strategy) .withRules(DS.WIKI, Load.on(Tier.T1, 1).forever()) .withServers(historicals) @@ -77,7 +76,6 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest loadQueuedSegments(); verifyValue(Metric.ASSIGNED_COUNT, 1000L); verifyNotEmitted(Metric.MOVED_COUNT); - verifyNotEmitted(Metric.UNMOVED_COUNT); for (DruidServer historical : historicals) { Assert.assertEquals(200, historical.getTotalSegments()); @@ -86,9 +84,8 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest // Run 2: nothing is assigned, nothing is moved as servers are already balanced runCoordinatorCycle(); loadQueuedSegments(); - verifyValue(Metric.ASSIGNED_COUNT, 0L); - verifyValue(Metric.MOVED_COUNT, 0L); - verifyValue(Metric.UNMOVED_COUNT, 1000L); + verifyNotEmitted(Metric.ASSIGNED_COUNT); + verifyNotEmitted(Metric.MOVED_COUNT); } @Test @@ -101,7 +98,6 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest CoordinatorSimulation sim = CoordinatorSimulation.builder() - .withDynamicConfig(createDynamicConfig(1000, 0, 100)) .withBalancer(strategy) .withRules(DS.WIKI, Load.on(Tier.T1, 1).forever()) .withServers(historicals) @@ -114,7 +110,6 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest loadQueuedSegments(); verifyValue(Metric.ASSIGNED_COUNT, 1000L); verifyNotEmitted(Metric.MOVED_COUNT); - verifyNotEmitted(Metric.UNMOVED_COUNT); // Verify that each server is equally loaded for (DruidServer historical : historicals) { @@ -127,7 +122,7 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest historicals.add(newHistorical); // Run the coordinator for a few cycles - for (int i = 0; i < 10; ++i) { + for (int i = 0; i < 7; ++i) { runCoordinatorCycle(); loadQueuedSegments(); } @@ -135,7 +130,7 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest // Verify that the segments have been balanced for (DruidServer historical : historicals) { long loadedSegments = historical.getTotalSegments(); - Assert.assertTrue(loadedSegments >= 199 && loadedSegments <= 201); + Assert.assertTrue(loadedSegments >= 195 && loadedSegments <= 205); } } @@ -149,7 +144,6 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest CoordinatorSimulation sim = CoordinatorSimulation.builder() - .withDynamicConfig(createDynamicConfig(1000, 0, 100)) .withBalancer(strategy) .withRules(DS.WIKI, Load.on(Tier.T1, 1).forever()) .withServers(historicals) @@ -162,7 +156,6 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest loadQueuedSegments(); verifyValue(Metric.ASSIGNED_COUNT, 1000L); verifyNotEmitted(Metric.MOVED_COUNT); - verifyNotEmitted(Metric.UNMOVED_COUNT); // Verify that each server is equally loaded for (DruidServer historical : historicals) { @@ -183,4 +176,5 @@ public class BalancingStrategiesTest extends CoordinatorSimulationBaseTest Assert.assertEquals(250, historical.getTotalSegments()); } } + } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java index 84c6886d9d4..99d4d9d0a83 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java @@ -25,13 +25,16 @@ import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.ForeverLoadRule; import org.apache.druid.server.coordinator.rules.Rule; +import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.timeline.DataSegment; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -158,43 +161,27 @@ public abstract class CoordinatorSimulationBaseTest implements // Utility methods - /** - * Creates a {@link CoordinatorDynamicConfig} with the specified values of: - * {@code maxSegmentsToMove, maxSegmentsInNodeLoadingQueue and replicationThrottleLimit}. - * The created config always has {@code useBatchedSegmentSampler=true} to avoid - * flakiness in tests. - * - * @see CoordinatorSimulationBaseTest - */ - static CoordinatorDynamicConfig createDynamicConfig( - int maxSegmentsToMove, - int maxSegmentsInNodeLoadingQueue, - int replicationThrottleLimit - ) + static Map filterByServer(DruidServer server) { - return CoordinatorDynamicConfig.builder() - .withMaxSegmentsToMove(maxSegmentsToMove) - .withReplicationThrottleLimit(replicationThrottleLimit) - .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInNodeLoadingQueue) - .withUseBatchedSegmentSampler(true) - .build(); + return filter(Dimension.SERVER, server.getName()); + } + + static Map filterByTier(String tier) + { + return filter(Dimension.TIER, tier); + } + + static Map filterByDatasource(String datasource) + { + return filter(Dimension.DATASOURCE, datasource); } /** * Creates a map containing dimension key-values to filter out metric events. */ - static Map filter(String... dimensionValues) + static Map filter(Dimension dimension, String value) { - if (dimensionValues.length < 2 || dimensionValues.length % 2 == 1) { - throw new IllegalArgumentException("Dimension key-values must be specified in pairs."); - } - - final Map filters = new HashMap<>(); - for (int i = 0; i < dimensionValues.length; ) { - filters.put(dimensionValues[i], dimensionValues[i + 1]); - i += 2; - } - return filters; + return Collections.singletonMap(dimension.reportedName(), value); } /** @@ -204,7 +191,7 @@ public abstract class CoordinatorSimulationBaseTest implements static DruidServer createHistorical(int uniqueIdInTier, String tier, long serverSizeMb) { final String name = tier + "__" + "hist__" + uniqueIdInTier; - return new DruidServer(name, name, name, serverSizeMb, ServerType.HISTORICAL, tier, 1); + return new DruidServer(name, name, name, serverSizeMb << 20, ServerType.HISTORICAL, tier, 1); } // Utility and constant holder classes @@ -219,16 +206,16 @@ public abstract class CoordinatorSimulationBaseTest implements { static final String T1 = "tier_t1"; static final String T2 = "tier_t2"; - static final String T3 = "tier_t3"; } static class Metric { static final String ASSIGNED_COUNT = "segment/assigned/count"; static final String MOVED_COUNT = "segment/moved/count"; - static final String UNMOVED_COUNT = "segment/unmoved/count"; static final String DROPPED_COUNT = "segment/dropped/count"; static final String LOAD_QUEUE_COUNT = "segment/loadQueue/count"; + static final String DROP_QUEUE_COUNT = "segment/dropQueue/count"; + static final String CANCELLED_ACTIONS = "segment/loadQueue/cancelled"; } static class Segments @@ -292,4 +279,15 @@ public abstract class CoordinatorSimulationBaseTest implements return new ForeverLoadRule(tieredReplicants); } } + + /** + * Builder for a broadcast rule. + */ + static class Broadcast + { + static Rule forever() + { + return new ForeverBroadcastDistributionRule(); + } + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 054d34422d9..bd9be366ae7 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -35,21 +35,22 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.server.coordinator.BalancerStrategyFactory; -import org.apache.druid.server.coordinator.CachingCostBalancerStrategyConfig; -import org.apache.druid.server.coordinator.CachingCostBalancerStrategyFactory; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; -import org.apache.druid.server.coordinator.CostBalancerStrategyFactory; -import org.apache.druid.server.coordinator.DiskNormalizedCostBalancerStrategyFactory; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; -import org.apache.druid.server.coordinator.LoadQueueTaskMaster; -import org.apache.druid.server.coordinator.RandomBalancerStrategyFactory; import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; +import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; +import org.apache.druid.server.coordinator.balancer.CachingCostBalancerStrategyConfig; +import org.apache.druid.server.coordinator.balancer.CachingCostBalancerStrategyFactory; +import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory; +import org.apache.druid.server.coordinator.balancer.DiskNormalizedCostBalancerStrategyFactory; +import org.apache.druid.server.coordinator.balancer.RandomBalancerStrategyFactory; import org.apache.druid.server.coordinator.duty.CompactionSegmentSearchPolicy; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups; import org.apache.druid.server.coordinator.duty.NewestSegmentFirstPolicy; +import org.apache.druid.server.coordinator.loading.LoadQueueTaskMaster; +import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.server.lookup.cache.LookupCoordinatorManager; import org.apache.druid.timeline.DataSegment; @@ -83,10 +84,7 @@ public class CoordinatorSimulationBuilder private static final CompactionSegmentSearchPolicy COMPACTION_SEGMENT_SEARCH_POLICY = new NewestSegmentFirstPolicy(OBJECT_MAPPER); private String balancerStrategy; - private CoordinatorDynamicConfig dynamicConfig = - CoordinatorDynamicConfig.builder() - .withUseBatchedSegmentSampler(true) - .build(); + private CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig.builder().build(); private List servers; private List segments; private final Map> datasourceRules = new HashMap<>(); @@ -156,7 +154,7 @@ public class CoordinatorSimulationBuilder * Default values: {@code useBatchedSegmentSampler = true}, other params as * specified in {@link CoordinatorDynamicConfig.Builder}. *

- * Tests that verify balancing behaviour should set + * Tests that verify balancing behaviour must set * {@link CoordinatorDynamicConfig#useBatchedSegmentSampler()} to true. * Otherwise, the segment sampling is random and can produce repeated values * leading to flakiness in the tests. The simulation sets this field to true by @@ -205,6 +203,7 @@ public class CoordinatorSimulationBuilder env.executorFactory, null, env.loadQueueTaskMaster, + env.loadQueueManager, new ServiceAnnouncer.Noop(), null, Collections.emptySet(), @@ -391,7 +390,7 @@ public class CoordinatorSimulationBuilder @Override public double getLoadPercentage(String datasource) { - return coordinator.getLoadStatus().get(datasource); + return coordinator.getDatasourceToLoadStatus().get(datasource); } @Override @@ -419,6 +418,7 @@ public class CoordinatorSimulationBuilder private final TestMetadataRuleManager ruleManager = new TestMetadataRuleManager(); private final LoadQueueTaskMaster loadQueueTaskMaster; + private final SegmentLoadQueueManager loadQueueManager; /** * Represents the current inventory of all servers (typically historicals) @@ -479,6 +479,8 @@ public class CoordinatorSimulationBuilder httpClient, null ); + this.loadQueueManager = + new SegmentLoadQueueManager(coordinatorInventoryView, segmentManager, loadQueueTaskMaster); this.jacksonConfigManager = mockConfigManager(); setDynamicConfig(dynamicConfig); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/RoundRobinAssignmentTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/RoundRobinAssignmentTest.java index cec100691a4..0fe884f58eb 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/RoundRobinAssignmentTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/RoundRobinAssignmentTest.java @@ -102,7 +102,8 @@ public class RoundRobinAssignmentTest extends CoordinatorSimulationBaseTest // Run 1: all segments are assigned and loaded runCoordinatorCycle(); loadQueuedSegments(); - verifyValue(Metric.ASSIGNED_COUNT, 13000L); + verifyValue(Metric.ASSIGNED_COUNT, filterByDatasource(DS.KOALA), 10000L); + verifyValue(Metric.ASSIGNED_COUNT, filterByDatasource(DS.WIKI), 3000L); for (DruidServer historical : historicals) { Assert.assertEquals(1300, historical.getTotalSegments()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentBalancingTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentBalancingTest.java index 1a90ee1eb25..383b5fe3c64 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentBalancingTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentBalancingTest.java @@ -20,7 +20,6 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; -import org.apache.druid.query.DruidMetrics; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; @@ -61,16 +60,12 @@ public class SegmentBalancingTest extends CoordinatorSimulationBaseTest private void testBalancingWithAutoSyncInventory(boolean autoSyncInventory) { - // maxSegmentsToMove = 10, unlimited load queue, no replication - CoordinatorDynamicConfig dynamicConfig = createDynamicConfig(10, 0, 0); - // historicals = 2(T1), replicas = 1(T1) final CoordinatorSimulation sim = CoordinatorSimulation.builder() .withSegments(segments) .withServers(historicalT11, historicalT12) .withRules(datasource, Load.on(Tier.T1, 1).forever()) - .withDynamicConfig(dynamicConfig) .withAutoInventorySync(autoSyncInventory) .build(); @@ -97,16 +92,12 @@ public class SegmentBalancingTest extends CoordinatorSimulationBaseTest @Test public void testDropDoesNotHappenWhenLoadFails() { - // maxSegmentsToMove = 10, unlimited load queue, no replication - CoordinatorDynamicConfig dynamicConfig = createDynamicConfig(10, 0, 0); - // historicals = 2(T1), replicas = 1(T1) final CoordinatorSimulation sim = CoordinatorSimulation.builder() .withSegments(segments) .withServers(historicalT11, historicalT12) .withRules(datasource, Load.on(Tier.T1, 1).forever()) - .withDynamicConfig(dynamicConfig) .build(); // Put all the segments on histT11 @@ -129,15 +120,11 @@ public class SegmentBalancingTest extends CoordinatorSimulationBaseTest @Test public void testBalancingOfFullyReplicatedSegment() { - // maxSegmentsToMove = 10, unlimited load queue, replicationThrottleLimit = 10 - CoordinatorDynamicConfig dynamicConfig = createDynamicConfig(10, 0, 10); - // historicals = 2(in T1), replicas = 1(T1) final CoordinatorSimulation sim = CoordinatorSimulation.builder() .withSegments(segments) .withServers(historicalT11, historicalT12) - .withDynamicConfig(dynamicConfig) .withRules(datasource, Load.on(Tier.T1, 1).forever()) .build(); @@ -149,21 +136,13 @@ public class SegmentBalancingTest extends CoordinatorSimulationBaseTest // Verify that there are segments in the load queue for balancing verifyValue(Metric.MOVED_COUNT, 5L); - verifyValue( - Metric.LOAD_QUEUE_COUNT, - filter(DruidMetrics.SERVER, historicalT12.getName()), - 5 - ); + verifyValue(Metric.LOAD_QUEUE_COUNT, filterByServer(historicalT12), 5L); runCoordinatorCycle(); // Verify that the segments in the load queue are not considered as over-replicated - verifyValue("segment/dropped/count", 0L); - verifyValue( - Metric.LOAD_QUEUE_COUNT, - filter(DruidMetrics.SERVER, historicalT12.getName()), - 5 - ); + verifyNotEmitted(Metric.DROPPED_COUNT); + verifyValue(Metric.LOAD_QUEUE_COUNT, filterByServer(historicalT12), 5L); // Finish and verify balancing loadQueuedSegments(); @@ -171,4 +150,83 @@ public class SegmentBalancingTest extends CoordinatorSimulationBaseTest Assert.assertEquals(5, historicalT12.getTotalSegments()); verifyDatasourceIsFullyLoaded(datasource); } + + @Test + public void testBalancingMovesSegmentsInLoadQueue() + { + CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(segments) + .withServers(historicalT11) + .withRules(datasource, Load.on(Tier.T1, 1).forever()) + .build(); + + startSimulation(sim); + + // Run 1: All segments are assigned to the first historical + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 10L); + verifyValue(Metric.LOAD_QUEUE_COUNT, filterByServer(historicalT11), 10L); + + // Run 2: Add new historical, some segments in the queue will be moved + addServer(historicalT12); + runCoordinatorCycle(); + verifyNotEmitted(Metric.ASSIGNED_COUNT); + verifyValue(Metric.CANCELLED_ACTIONS, 5L); + verifyValue(Metric.MOVED_COUNT, 5L); + + verifyValue(Metric.LOAD_QUEUE_COUNT, filterByServer(historicalT11), 5L); + verifyValue(Metric.LOAD_QUEUE_COUNT, filterByServer(historicalT12), 5L); + + // Complete loading the segments + loadQueuedSegments(); + Assert.assertEquals(5, historicalT11.getTotalSegments()); + Assert.assertEquals(5, historicalT12.getTotalSegments()); + } + + @Test + public void testBalancingDoesNotMoveLoadedSegmentsWhenTierIsBusy() + { + // maxSegmentsToMove = 3, unlimited load queue + final CoordinatorDynamicConfig dynamicConfig = + CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withMaxSegmentsToMove(3) + .withMaxSegmentsInNodeLoadingQueue(0) + .build(); + + CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withDynamicConfig(dynamicConfig) + .withSegments(segments) + .withServers(historicalT11) + .withRules(datasource, Load.on(Tier.T1, 1).forever()) + .build(); + + startSimulation(sim); + + // Pre-load some of the segments on histT11 + segments.subList(2, segments.size()).forEach(historicalT11::addDataSegment); + + // Run 1: The remaining segments are assigned to histT11 + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 2L); + + // Run 2: Add histT12, some loading segments and some loaded segments are moved to it + addServer(historicalT12); + runCoordinatorCycle(); + verifyValue(Metric.MOVED_COUNT, 3L); + verifyValue(Metric.CANCELLED_ACTIONS, 2L); + verifyValue(Metric.LOAD_QUEUE_COUNT, filterByServer(historicalT12), 3L); + + // Run 3: No more segments are moved as tier is already busy moving + runCoordinatorCycle(); + verifyNotEmitted(Metric.MOVED_COUNT); + + // Run 4: Load pending segments, more are moved + loadQueuedSegments(); + runCoordinatorCycle(); + Assert.assertTrue(getValue(Metric.MOVED_COUNT, null).intValue() > 0); + } + } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingNegativeTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingNegativeTest.java deleted file mode 100644 index b118819e3c4..00000000000 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingNegativeTest.java +++ /dev/null @@ -1,220 +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.druid.server.coordinator.simulate; - -import org.apache.druid.client.DruidServer; -import org.apache.druid.query.DruidMetrics; -import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; -import org.apache.druid.timeline.DataSegment; -import org.junit.Assert; -import org.junit.Test; - -import java.util.List; - -/** - * Contains negative tests that verify existing erroneous behaviour of segment - * loading. The underlying issues should be fixed and the modified tests - * should be migrated to {@link SegmentLoadingTest}. - *

- * Identified issues: - * Apache #12881 - */ -public class SegmentLoadingNegativeTest extends CoordinatorSimulationBaseTest -{ - private DruidServer historicalT11; - private DruidServer historicalT12; - private DruidServer historicalT21; - - private final String datasource = DS.WIKI; - private final List segments = Segments.WIKI_10X1D; - - @Override - public void setUp() - { - // Setup historicals for 2 tiers, size 10 GB each - historicalT11 = createHistorical(1, Tier.T1, 10_000); - historicalT12 = createHistorical(2, Tier.T1, 10_000); - historicalT21 = createHistorical(1, Tier.T2, 10_000); - } - - /** - * Correct behaviour: replicationThrottleLimit should not be violated even if - * segment loading is fast. - *

- * Fix Apache #12881 to fix this test. - */ - @Test - public void testImmediateLoadingViolatesThrottleLimit() - { - // Disable balancing, infinite load queue size, replicationThrottleLimit = 2 - CoordinatorDynamicConfig dynamicConfig = createDynamicConfig(0, 0, 2); - - // historicals = 2(in T1), segments = 10*1day - // replicas = 2(on T1), immediate segment loading - final CoordinatorSimulation sim = - CoordinatorSimulation.builder() - .withSegments(segments) - .withServers(historicalT11, historicalT12) - .withRules(datasource, Load.on(Tier.T1, 2).forever()) - .withImmediateSegmentLoading(true) - .withDynamicConfig(dynamicConfig) - .build(); - - // Put the first replica of all the segments on histT11 - segments.forEach(historicalT11::addDataSegment); - - startSimulation(sim); - runCoordinatorCycle(); - - // Verify that number of replicas assigned exceeds the replicationThrottleLimit - verifyValue(Metric.ASSIGNED_COUNT, 10L); - - Assert.assertEquals(10, historicalT11.getTotalSegments()); - Assert.assertEquals(10, historicalT12.getTotalSegments()); - verifyDatasourceIsFullyLoaded(datasource); - } - - /** - * Correct behaviour: The first replica on any tier should not be throttled. - *

- * Fix Apache #12881 to fix this test. - */ - @Test - public void testFirstReplicaOnAnyTierIsThrottled() - { - // Disable balancing, infinite load queue size, replicateThrottleLimit = 2 - CoordinatorDynamicConfig dynamicConfig = createDynamicConfig(0, 0, 2); - - // historicals = 1(in T1) + 1(in T2) - // replicas = 1(on T1) + 1(on T2) - final CoordinatorSimulation sim = - CoordinatorSimulation.builder() - .withSegments(segments) - .withServers(historicalT11, historicalT21) - .withDynamicConfig(dynamicConfig) - .withRules( - datasource, - Load.on(Tier.T1, 1).andOn(Tier.T2, 1).forever() - ) - .build(); - - // Put the first replica of all the segments on T1 - segments.forEach(historicalT11::addDataSegment); - - startSimulation(sim); - runCoordinatorCycle(); - - // Verify that num replicas assigned to T2 are equal to the replicationthrottleLimit - verifyValue( - Metric.ASSIGNED_COUNT, - filter(DruidMetrics.TIER, Tier.T2), - 2L - ); - - loadQueuedSegments(); - - verifyDatasourceIsFullyLoaded(datasource); - Assert.assertEquals(10, historicalT11.getTotalSegments()); - Assert.assertEquals(2, historicalT21.getTotalSegments()); - } - - /** - * Correct behaviour: Historical should not get overassigned even if loading is fast. - *

- * Fix Apache #12881 to fix this test. - */ - @Test - public void testImmediateLoadingOverassignsHistorical() - { - // historicals = 1(in T1), size 1 GB - final DruidServer historicalT11 = createHistorical(1, Tier.T1, 1000); - - // disable balancing, unlimited load queue, replicationThrottleLimit = 10 - CoordinatorDynamicConfig dynamicConfig = createDynamicConfig(0, 0, 10); - - // segments = 10*1day, size 500 MB - // strategy = cost, replicas = 1(T1) - final CoordinatorSimulation sim = - CoordinatorSimulation.builder() - .withSegments(segments) - .withServers(historicalT11) - .withDynamicConfig(dynamicConfig) - .withRules(datasource, Load.on(Tier.T1, 1).forever()) - .withImmediateSegmentLoading(true) - .build(); - - startSimulation(sim); - runCoordinatorCycle(); - - // The historical is assigned several segments but loads only upto its capacity - verifyValue(Metric.ASSIGNED_COUNT, 10L); - Assert.assertEquals(2, historicalT11.getTotalSegments()); - } - - /** - * Correct behaviour: For a fully replicated segment, items that are in the load - * queue should get cancelled so that the coordinator does not have to wait - * for the loads to finish and then take remedial action. - *

- * Fix Apache #12881 to fix this test case. - */ - @Test - public void testLoadOfFullyReplicatedSegmentIsNotCancelled() - { - // disable balancing, unlimited load queue, replicationThrottleLimit = 10 - CoordinatorDynamicConfig dynamicConfig = createDynamicConfig(0, 0, 10); - - // historicals = 2(in T1), replicas = 2(on T1) - final CoordinatorSimulation sim = - CoordinatorSimulation.builder() - .withSegments(segments) - .withServers(historicalT11, historicalT12) - .withDynamicConfig(dynamicConfig) - .withRules(datasource, Load.on(Tier.T1, 2).forever()) - .build(); - - // Put the first replica of all the segments on histT11 - segments.forEach(historicalT11::addDataSegment); - - startSimulation(sim); - runCoordinatorCycle(); - - // Verify that there are segments in the load queue - verifyValue(Metric.ASSIGNED_COUNT, 10L); - verifyValue( - Metric.LOAD_QUEUE_COUNT, - filter(DruidMetrics.SERVER, historicalT12.getName()), - 10 - ); - - // Put the second replica of all the segments on histT12 - segments.forEach(historicalT12::addDataSegment); - - runCoordinatorCycle(); - - // Verify that the segments are still in the load queue - verifyValue( - Metric.LOAD_QUEUE_COUNT, - filter(DruidMetrics.SERVER, historicalT12.getName()), - 10 - ); - } - -} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java index e04ebe532df..f0ea5cc9ae5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java @@ -20,12 +20,12 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; -import org.apache.druid.query.DruidMetrics; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Test; +import java.util.Collections; import java.util.List; /** @@ -35,6 +35,7 @@ public class SegmentLoadingTest extends CoordinatorSimulationBaseTest { private DruidServer historicalT11; private DruidServer historicalT12; + private DruidServer historicalT13; private DruidServer historicalT21; private DruidServer historicalT22; @@ -47,6 +48,7 @@ public class SegmentLoadingTest extends CoordinatorSimulationBaseTest // Setup historicals for 2 tiers, size 10 GB each historicalT11 = createHistorical(1, Tier.T1, 10_000); historicalT12 = createHistorical(2, Tier.T1, 10_000); + historicalT13 = createHistorical(3, Tier.T1, 10_000); historicalT21 = createHistorical(1, Tier.T2, 10_000); historicalT22 = createHistorical(2, Tier.T2, 10_000); @@ -55,9 +57,6 @@ public class SegmentLoadingTest extends CoordinatorSimulationBaseTest @Test public void testSecondReplicaOnAnyTierIsThrottled() { - // Disable balancing, infinite load queue size, replicateThrottleLimit = 2 - CoordinatorDynamicConfig dynamicConfig = createDynamicConfig(0, 0, 2); - // historicals = 2(in T1) // replicas = 2(on T1) final CoordinatorSimulation sim = @@ -65,7 +64,7 @@ public class SegmentLoadingTest extends CoordinatorSimulationBaseTest .withSegments(segments) .withServers(historicalT11, historicalT12) .withRules(datasource, Load.on(Tier.T1, 2).forever()) - .withDynamicConfig(dynamicConfig) + .withDynamicConfig(withReplicationThrottleLimit(2)) .build(); // Put the first replica of all the segments on histT11 @@ -88,16 +87,13 @@ public class SegmentLoadingTest extends CoordinatorSimulationBaseTest // historicals = 1(in T1), size 1 GB final DruidServer historicalT11 = createHistorical(1, Tier.T1, 1000); - // disable balancing, unlimited load queue, replicationThrottleLimit = 10 - CoordinatorDynamicConfig dynamicConfig = createDynamicConfig(0, 0, 10); - // segments = 10*1day, size 500 MB // strategy = cost, replicas = 1(T1) final CoordinatorSimulation sim = CoordinatorSimulation.builder() .withSegments(segments) .withServers(historicalT11) - .withDynamicConfig(dynamicConfig) + .withDynamicConfig(withReplicationThrottleLimit(10)) .withRules(datasource, Load.on(Tier.T1, 1).forever()) .withImmediateSegmentLoading(false) .build(); @@ -112,81 +108,432 @@ public class SegmentLoadingTest extends CoordinatorSimulationBaseTest } @Test - public void testDropHappensAfterTargetReplicationOnEveryTier() + public void testTierShiftDoesNotCauseUnderReplication() { - // maxNonPrimaryReplicants = 33 ensures that all target replicas (total 4) - // are assigned for some segments in the first run itself (pigeon-hole) - CoordinatorDynamicConfig dynamicConfig = - CoordinatorDynamicConfig.builder() - .withMaxSegmentsToMove(0) - .withReplicationThrottleLimit(10) - .withMaxNonPrimaryReplicantsToLoad(33) - .build(); - - // historicals = 1(in T1) + 2(in T2) + 2(in T3) - // segments = 10 * 1day, replicas = 2(T2) + 2(T3) - final DruidServer historicalT31 = createHistorical(1, Tier.T3, 10_000); - final DruidServer historicalT32 = createHistorical(2, Tier.T3, 10_000); + // historicals = 2(in T1) + 3(in T2) + // segments = 1, replicas = 3(T2) + final DataSegment segment = segments.get(0); final CoordinatorSimulation sim = CoordinatorSimulation.builder() - .withSegments(segments) - .withDynamicConfig(dynamicConfig) - .withRules(datasource, Load.on(Tier.T2, 2).andOn(Tier.T3, 2).forever()) - .withServers( - historicalT11, - historicalT21, - historicalT22, - historicalT31, - historicalT32 - ) + .withSegments(Collections.singletonList(segment)) + .withDynamicConfig(withReplicationThrottleLimit(10)) + .withRules(datasource, Load.on(Tier.T2, 3).forever()) + .withServers(historicalT11, historicalT12, historicalT21, historicalT22) .build(); - // At the start, T1 has all the segments - segments.forEach(historicalT11::addDataSegment); + // At the start, T1 has the segment fully replicated + historicalT11.addDataSegment(segment); + historicalT12.addDataSegment(segment); - // Run 1: Nothing is dropped from T1 but things are assigned to T2 and T3 + // Run 1: Nothing is dropped from T1 but 2 replicas are assigned to T2 startSimulation(sim); runCoordinatorCycle(); verifyNotEmitted(Metric.DROPPED_COUNT); - int totalAssignedInRun1 - = getValue(Metric.ASSIGNED_COUNT, filter(DruidMetrics.TIER, Tier.T2)).intValue() - + getValue(Metric.ASSIGNED_COUNT, filter(DruidMetrics.TIER, Tier.T3)).intValue(); - Assert.assertTrue(totalAssignedInRun1 > 0 && totalAssignedInRun1 < 40); + verifyValue(Metric.ASSIGNED_COUNT, filterByTier(Tier.T2), 2L); - // Run 2: Segments still queued, nothing is dropped from T1 + // Run 2: Replicas still queued + // nothing new is assigned to T2, nothing is dropped from T1 runCoordinatorCycle(); - loadQueuedSegments(); verifyNotEmitted(Metric.DROPPED_COUNT); - int totalLoadedAfterRun2 - = historicalT21.getTotalSegments() + historicalT22.getTotalSegments() - + historicalT31.getTotalSegments() + historicalT32.getTotalSegments(); - Assert.assertEquals(totalAssignedInRun1, totalLoadedAfterRun2); + verifyValue(Metric.ASSIGNED_COUNT, filterByTier(Tier.T2), 0L); - // Run 3: Some segments have been loaded - // segments fully replicated on T2 and T3 will now be dropped from T1 - runCoordinatorCycle(); loadQueuedSegments(); + Assert.assertEquals(2, getNumLoadedSegments(historicalT21, historicalT22)); + Assert.assertEquals(2, getNumLoadedSegments(historicalT11, historicalT12)); - int totalDroppedInRun3 - = getValue(Metric.DROPPED_COUNT, filter(DruidMetrics.TIER, Tier.T1)).intValue(); - Assert.assertTrue(totalDroppedInRun3 > 0 && totalDroppedInRun3 < 10); - int totalLoadedAfterRun3 - = historicalT21.getTotalSegments() + historicalT22.getTotalSegments() - + historicalT31.getTotalSegments() + historicalT32.getTotalSegments(); - Assert.assertEquals(40, totalLoadedAfterRun3); - - // Run 4: All segments are fully replicated on T2 and T3 + // Run 3: total loaded replicas (4) > total required replicas (3) > total loadable replicas (2) + // no server to assign third replica in T2, but all replicas are dropped from T1 runCoordinatorCycle(); + + verifyValue(Metric.DROPPED_COUNT, filterByTier(Tier.T1), 2L); + verifyValue(Metric.ASSIGNED_COUNT, filterByTier(Tier.T2), 0L); + loadQueuedSegments(); + Assert.assertEquals(2, getNumLoadedSegments(historicalT21, historicalT22)); + Assert.assertEquals(0, getNumLoadedSegments(historicalT11, historicalT12)); - int totalDroppedInRun4 - = getValue(Metric.DROPPED_COUNT, filter(DruidMetrics.TIER, Tier.T1)).intValue(); + // Run 4: Add 3rd server to T2, third replica can now be assigned + // Add 3rd server to T1 with replica loaded, but it will not be dropped + final DruidServer historicalT23 = createHistorical(3, Tier.T2, 10_000); + addServer(historicalT23); + historicalT13.addDataSegment(segment); + addServer(historicalT13); + runCoordinatorCycle(); - Assert.assertEquals(10, totalDroppedInRun3 + totalDroppedInRun4); - Assert.assertEquals(0, historicalT11.getTotalSegments()); + verifyNotEmitted(Metric.DROPPED_COUNT); + verifyValue(Metric.ASSIGNED_COUNT, filterByTier(Tier.T2), 1L); + + loadQueuedSegments(); + Assert.assertEquals(3, getNumLoadedSegments(historicalT21, historicalT22, historicalT23)); + Assert.assertEquals(1, historicalT13.getTotalSegments()); + + // Run 5: segment is fully replicated on T2, remaining replica will now be dropped from T1 + runCoordinatorCycle(); + + verifyValue(Metric.DROPPED_COUNT, filterByTier(Tier.T1), 1L); + verifyNotEmitted(Metric.ASSIGNED_COUNT); + + loadQueuedSegments(); + Assert.assertEquals(3, getNumLoadedSegments(historicalT21, historicalT22, historicalT23)); + Assert.assertEquals(0, getNumLoadedSegments(historicalT11, historicalT12, historicalT13)); verifyDatasourceIsFullyLoaded(datasource); } + @Test + public void testTierAddDoesNotCauseUnderReplication() + { + // historicals = 2(in T1) + 1(in T2) + // current replicas = 2(T1) + // required replicas = 1(T1) + 1(T2) + final DataSegment segment = segments.get(0); + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(Collections.singletonList(segment)) + .withDynamicConfig(withReplicationThrottleLimit(10)) + .withRules(datasource, Load.on(Tier.T1, 1).andOn(Tier.T2, 1).forever()) + .withServers(historicalT11, historicalT12, historicalT21) + .build(); + + // At the start, T1 has 2 replicas of the segment + historicalT11.addDataSegment(segment); + historicalT12.addDataSegment(segment); + + // Run 1: Nothing is dropped from T1 but 1 replica is assigned to T2 + startSimulation(sim); + runCoordinatorCycle(); + + verifyNotEmitted(Metric.DROPPED_COUNT); + verifyValue(Metric.ASSIGNED_COUNT, filterByTier(Tier.T2), 1L); + + // Run 2: Replicas still queued + // nothing new is assigned to T2, nothing is dropped from T1 + runCoordinatorCycle(); + + verifyNotEmitted(Metric.DROPPED_COUNT); + verifyNotEmitted(Metric.ASSIGNED_COUNT); + + loadQueuedSegments(); + Assert.assertEquals(1, getNumLoadedSegments(historicalT21)); + Assert.assertEquals(2, getNumLoadedSegments(historicalT11, historicalT12)); + + // Run 3: total loaded replicas (3) > total required replicas (2) + // one replica is dropped from T1 + runCoordinatorCycle(); + + verifyValue(Metric.DROPPED_COUNT, filterByTier(Tier.T1), 1L); + + loadQueuedSegments(); + Assert.assertEquals(1, getNumLoadedSegments(historicalT21)); + Assert.assertEquals(1, getNumLoadedSegments(historicalT11, historicalT12)); + } + + @Test + public void testImmediateLoadingDoesNotOverassignHistorical() + { + // historicals = 1(in T1), size 1 GB + final DruidServer historicalT11 = createHistorical(1, Tier.T1, 1000); + + // segments = 10*1day, size 500 MB + // strategy = cost, replicas = 1(T1) + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(segments) + .withServers(historicalT11) + .withDynamicConfig(withReplicationThrottleLimit(10)) + .withRules(datasource, Load.on(Tier.T1, 1).forever()) + .withImmediateSegmentLoading(true) + .build(); + + startSimulation(sim); + runCoordinatorCycle(); + + // The historical is only assigned segments that it can load + verifyValue(Metric.ASSIGNED_COUNT, 2L); + Assert.assertEquals(2, historicalT11.getTotalSegments()); + } + + @Test + public void testMaxSegmentsInNodeLoadingQueue() + { + // disable balancing, maxSegmentsInNodeLoadingQueue = 5, replicationThrottleLimit = 10 + CoordinatorDynamicConfig dynamicConfig = + CoordinatorDynamicConfig.builder() + .withMaxSegmentsToMove(0) + .withReplicationThrottleLimit(10) + .withMaxSegmentsInNodeLoadingQueue(5) + .withUseRoundRobinSegmentAssignment(false) + .withSmartSegmentLoading(false) + .build(); + + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(segments) + .withServers(historicalT11) + .withDynamicConfig(dynamicConfig) + .withRules(datasource, Load.on(Tier.T1, 1).forever()) + .build(); + + startSimulation(sim); + + // Run 1: Only some segments are assigned as load queue size is limited + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 5L); + + // Run 2: No more segments are assigned as queue is already full + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 0L); + + // Instantly load some of the queued segments on the historical + historicalT11.addDataSegment(segments.get(9)); + historicalT11.addDataSegment(segments.get(8)); + + // Run 3: No segments are assigned, extra loads are cancelled + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 0L); + verifyValue(Metric.CANCELLED_ACTIONS, 2L); + + // Run 4: Some segments are assigned as load queue is still partially full + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 2L); + } + + @Test + public void testFirstReplicaOnTierIsNotThrottled() + { + // historicals = 1(in T1) + 1(in T2) + // replicas = 1(on T1) + 1(on T2) + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(segments) + .withServers(historicalT11, historicalT21) + .withDynamicConfig(withReplicationThrottleLimit(2)) + .withRules(datasource, Load.on(Tier.T1, 1).andOn(Tier.T2, 1).forever()) + .build(); + + // Put the first replica of all the segments on T1 + segments.forEach(historicalT11::addDataSegment); + + startSimulation(sim); + runCoordinatorCycle(); + + // Verify that primary replica on T2 are not throttled + verifyValue(Metric.ASSIGNED_COUNT, filterByTier(Tier.T2), 10L); + + loadQueuedSegments(); + + verifyDatasourceIsFullyLoaded(datasource); + Assert.assertEquals(10, historicalT11.getTotalSegments()); + Assert.assertEquals(10, historicalT21.getTotalSegments()); + } + + @Test + public void testImmediateLoadingDoesNotViolateThrottleLimit() + { + // historicals = 2(in T1), segments = 10*1day + // replicas = 2(on T1), immediate segment loading + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(segments) + .withServers(historicalT11, historicalT12) + .withRules(datasource, Load.on(Tier.T1, 2).forever()) + .withImmediateSegmentLoading(true) + .withDynamicConfig(withReplicationThrottleLimit(2)) + .build(); + + // Put the first replica of all the segments on histT11 + segments.forEach(historicalT11::addDataSegment); + + startSimulation(sim); + runCoordinatorCycle(); + + // Verify that number of replicas does not exceed the replicationThrottleLimit + verifyValue(Metric.ASSIGNED_COUNT, 2L); + + Assert.assertEquals(10, historicalT11.getTotalSegments()); + Assert.assertEquals(2, historicalT12.getTotalSegments()); + } + + @Test + public void testLoadOfFullyReplicatedSegmentGetsCancelled() + { + // historicals = 2(in T1), replicas = 2(on T1) + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(segments) + .withServers(historicalT11, historicalT12) + .withDynamicConfig(withReplicationThrottleLimit(10)) + .withRules(datasource, Load.on(Tier.T1, 2).forever()) + .build(); + + // Put the first replica of all the segments on histT11 + segments.forEach(historicalT11::addDataSegment); + + startSimulation(sim); + runCoordinatorCycle(); + + // Verify that there are segments in the load queue + verifyValue(Metric.ASSIGNED_COUNT, 10L); + verifyValue( + Metric.LOAD_QUEUE_COUNT, + filterByServer(historicalT12), + 10L + ); + + // Add a new historical with the second replica of all the segments + addServer(historicalT13); + segments.forEach(historicalT13::addDataSegment); + + runCoordinatorCycle(); + + // Verify that the loading of the extra replicas is cancelled + verifyValue(Metric.CANCELLED_ACTIONS, 10L); + verifyValue(Metric.LOAD_QUEUE_COUNT, filterByServer(historicalT12), 0L); + } + + @Test + public void testBroadcastReplicasAreNotThrottled() + { + // historicals = 3(in T1) + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(segments) + .withServers(historicalT11, historicalT12, historicalT13) + .withDynamicConfig(withReplicationThrottleLimit(0)) + .withRules(datasource, Broadcast.forever()) + .build(); + + startSimulation(sim); + runCoordinatorCycle(); + + // Verify that all the segments are broadcast to all historicals + // irrespective of throttle limit + verifyValue(Metric.ASSIGNED_COUNT, filterByDatasource(DS.WIKI), 30L); + verifyNotEmitted(Metric.DROPPED_COUNT); + } + + @Test + public void testReplicasAreNotAssignedIfTierIsBusy() + { + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(segments) + .withServers(historicalT11, historicalT12) + .withDynamicConfig(withReplicationThrottleLimit(5)) + .withRules(datasource, Load.on(Tier.T1, 2).forever()) + .build(); + + startSimulation(sim); + + // All segments are loaded on histT11 + segments.forEach(historicalT11::addDataSegment); + + // Run 1: Some replicas are assigned to histT12 + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 5L); + + // Run 2: No more replicas are assigned because tier is already busy with replication + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 0L); + + // Run 3: Remaining replicas are assigned + loadQueuedSegments(); + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 5L); + + Assert.assertEquals(10, historicalT11.getTotalSegments()); + Assert.assertEquals(5, historicalT12.getTotalSegments()); + } + + @Test + public void testAllLoadsOnDecommissioningServerAreCancelled() + { + final CoordinatorDynamicConfig dynamicConfig = withReplicationThrottleLimit(100); + + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withSegments(segments) + .withServers(historicalT11, historicalT12) + .withDynamicConfig(dynamicConfig) + .withRules(datasource, Load.on(Tier.T1, 2).forever()) + .build(); + + startSimulation(sim); + + // All segments are loaded on histT11 + segments.forEach(historicalT11::addDataSegment); + + // Run 1: Some replicas are assigned to histT12 + runCoordinatorCycle(); + verifyValue(Metric.ASSIGNED_COUNT, 10L); + + // Run 2: histT12 is marked as decommissioning, all loads on it are cancelled + setDynamicConfig( + CoordinatorDynamicConfig.builder() + .withDecommissioningNodes(Collections.singleton(historicalT12.getName())) + .build(dynamicConfig) + ); + runCoordinatorCycle(); + verifyValue(Metric.CANCELLED_ACTIONS, 10L); + + loadQueuedSegments(); + Assert.assertEquals(10, historicalT11.getTotalSegments()); + Assert.assertEquals(0, historicalT12.getTotalSegments()); + } + + @Test + public void testSegmentsAreDroppedFromFullServersFirst() + { + final CoordinatorSimulation sim = + CoordinatorSimulation.builder() + .withServers(historicalT11, historicalT12) + .withDynamicConfig(withReplicationThrottleLimit(100)) + .withRules(datasource, Load.on(Tier.T1, 1).forever()) + .withRules(DS.KOALA, Load.on(Tier.T1, 1).forever()) + .build(); + + startSimulation(sim); + + // All wiki segments are loaded on both historicals + addSegments(segments); + segments.forEach(historicalT11::addDataSegment); + segments.forEach(historicalT12::addDataSegment); + + // Add extra koala segments to histT11 + final List koalaSegments = Segments.KOALA_100X100D.subList(0, 2); + addSegments(koalaSegments); + koalaSegments.forEach(historicalT11::addDataSegment); + + // More segments are dropped from histT11 as it was more full before the run + runCoordinatorCycle(); + verifyValue(Metric.DROP_QUEUE_COUNT, filterByServer(historicalT11), 6L); + verifyValue(Metric.DROP_QUEUE_COUNT, filterByServer(historicalT12), 4L); + + loadQueuedSegments(); + Assert.assertEquals(historicalT11.getCurrSize(), historicalT12.getCurrSize()); + } + + private int getNumLoadedSegments(DruidServer... servers) + { + int numLoaded = 0; + for (DruidServer server : servers) { + numLoaded += server.getTotalSegments(); + } + return numLoaded; + } + + /** + * Creates a dynamic config with unlimited load queue, balancing disabled and + * the given {@code replicationThrottleLimit}. + */ + private CoordinatorDynamicConfig withReplicationThrottleLimit(int replicationThrottleLimit) + { + return CoordinatorDynamicConfig.builder() + .withSmartSegmentLoading(false) + .withMaxSegmentsToMove(0) + .withMaxSegmentsInNodeLoadingQueue(0) + .withReplicationThrottleLimit(replicationThrottleLimit) + .build(); + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java index e29c2ebbf8d..ac7f2dacb66 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java @@ -43,16 +43,20 @@ public class TestSegmentsMetadataManager implements SegmentsMetadataManager private final ConcurrentMap segments = new ConcurrentHashMap<>(); private final ConcurrentMap usedSegments = new ConcurrentHashMap<>(); + private volatile DataSourcesSnapshot snapshot; + public void addSegment(DataSegment segment) { segments.put(segment.getId().toString(), segment); usedSegments.put(segment.getId().toString(), segment); + snapshot = null; } public void removeSegment(DataSegment segment) { segments.remove(segment.getId().toString()); usedSegments.remove(segment.getId().toString()); + snapshot = null; } @Override @@ -136,7 +140,10 @@ public class TestSegmentsMetadataManager implements SegmentsMetadataManager @Override public ImmutableDruidDataSource getImmutableDataSourceWithUsedSegments(String dataSource) { - return null; + if (snapshot == null) { + getSnapshotOfDataSourcesWithAllUsedSegments(); + } + return snapshot.getDataSource(dataSource); } @Override @@ -148,7 +155,10 @@ public class TestSegmentsMetadataManager implements SegmentsMetadataManager @Override public DataSourcesSnapshot getSnapshotOfDataSourcesWithAllUsedSegments() { - return DataSourcesSnapshot.fromUsedSegments(usedSegments.values(), ImmutableMap.of()); + if (snapshot == null) { + snapshot = DataSourcesSnapshot.fromUsedSegments(usedSegments.values(), ImmutableMap.of()); + } + return snapshot; } @Override diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java index b09d0faec7c..2b81a5c918c 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java @@ -257,7 +257,9 @@ public class CoordinatorDynamicConfigTest true, true, 10, - false + false, + false, + null ); Assert.assertTrue(config.isKillUnusedSegmentsInAllDataSources()); Assert.assertTrue(config.getSpecificDataSourcesToKillUnusedSegmentsIn().isEmpty()); @@ -285,7 +287,9 @@ public class CoordinatorDynamicConfigTest true, true, 10, - false + false, + false, + null ); Assert.assertFalse(config.isKillUnusedSegmentsInAllDataSources()); Assert.assertEquals(ImmutableSet.of("test1"), config.getSpecificDataSourcesToKillUnusedSegmentsIn()); @@ -436,70 +440,21 @@ public class CoordinatorDynamicConfigTest } @Test - public void testSerdeHandleInvalidPercentOfSegmentsToConsiderPerMove() throws Exception + public void testSerdeHandleInvalidPercentOfSegmentsToConsiderPerMove() { - try { - String jsonStr = "{\n" - + " \"percentOfSegmentsToConsiderPerMove\": 0\n" - + "}\n"; - - mapper.readValue( - mapper.writeValueAsString( - mapper.readValue( - jsonStr, - CoordinatorDynamicConfig.class - ) - ), - CoordinatorDynamicConfig.class - ); - - Assert.fail("deserialization should fail."); - } - catch (JsonMappingException e) { - Assert.assertTrue(e.getCause() instanceof IllegalArgumentException); - } - - try { - String jsonStr = "{\n" - + " \"percentOfSegmentsToConsiderPerMove\": -100\n" - + "}\n"; - - mapper.readValue( - mapper.writeValueAsString( - mapper.readValue( - jsonStr, - CoordinatorDynamicConfig.class - ) - ), - CoordinatorDynamicConfig.class - ); - - Assert.fail("deserialization should fail."); - } - catch (JsonMappingException e) { - Assert.assertTrue(e.getCause() instanceof IllegalArgumentException); - } - - try { - String jsonStr = "{\n" - + " \"percentOfSegmentsToConsiderPerMove\": 105\n" - + "}\n"; - - mapper.readValue( - mapper.writeValueAsString( - mapper.readValue( - jsonStr, - CoordinatorDynamicConfig.class - ) - ), - CoordinatorDynamicConfig.class - ); - - Assert.fail("deserialization should fail."); - } - catch (JsonMappingException e) { - Assert.assertTrue(e.getCause() instanceof IllegalArgumentException); - } + final String errorMsg = "'percentOfSegmentsToConsiderPerMove' should be between 1 and 100"; + assertThatDeserializationFailsWithMessage( + "{\"percentOfSegmentsToConsiderPerMove\": 0}", + errorMsg + ); + assertThatDeserializationFailsWithMessage( + "{\"percentOfSegmentsToConsiderPerMove\": -100}", + errorMsg + ); + assertThatDeserializationFailsWithMessage( + "{\"percentOfSegmentsToConsiderPerMove\": 105}", + errorMsg + ); } @Test @@ -522,10 +477,7 @@ public class CoordinatorDynamicConfigTest + "}\n"; CoordinatorDynamicConfig actual = mapper.readValue( mapper.writeValueAsString( - mapper.readValue( - jsonStr, - CoordinatorDynamicConfig.class - ) + mapper.readValue(jsonStr, CoordinatorDynamicConfig.class) ), CoordinatorDynamicConfig.class ); @@ -573,10 +525,7 @@ public class CoordinatorDynamicConfigTest CoordinatorDynamicConfig actual = mapper.readValue( mapper.writeValueAsString( - mapper.readValue( - jsonStr, - CoordinatorDynamicConfig.class - ) + mapper.readValue(jsonStr, CoordinatorDynamicConfig.class) ), CoordinatorDynamicConfig.class ); @@ -611,10 +560,7 @@ public class CoordinatorDynamicConfigTest + " \"killAllDataSources\": true,\n" + " \"percentOfSegmentsToConsiderPerMove\": 1\n" + "}\n"; - actual = mapper.readValue( - jsonStr, - CoordinatorDynamicConfig.class - ); + actual = mapper.readValue(jsonStr, CoordinatorDynamicConfig.class); Assert.assertFalse(actual.isKillUnusedSegmentsInAllDataSources()); Assert.assertEquals(2, actual.getSpecificDataSourcesToKillUnusedSegmentsIn().size()); @@ -638,10 +584,7 @@ public class CoordinatorDynamicConfigTest CoordinatorDynamicConfig actual = mapper.readValue( mapper.writeValueAsString( - mapper.readValue( - jsonStr, - CoordinatorDynamicConfig.class - ) + mapper.readValue(jsonStr, CoordinatorDynamicConfig.class) ), CoordinatorDynamicConfig.class ); @@ -757,34 +700,36 @@ public class CoordinatorDynamicConfigTest null, null, null, + null, + null, null ).build(current) ); } @Test - public void testSerdeHandleInvalidMaxNonPrimaryReplicantsToLoad() throws Exception + public void testSerdeHandleInvalidMaxNonPrimaryReplicantsToLoad() { - try { - String jsonStr = "{\n" - + " \"maxNonPrimaryReplicantsToLoad\": -1\n" - + "}\n"; + assertThatDeserializationFailsWithMessage( + "{\"maxNonPrimaryReplicantsToLoad\": -1}", + "maxNonPrimaryReplicantsToLoad must be greater than or equal to 0." + ); + } - mapper.readValue( - mapper.writeValueAsString( - mapper.readValue( - jsonStr, - CoordinatorDynamicConfig.class - ) - ), - CoordinatorDynamicConfig.class - ); - - Assert.fail("deserialization should fail."); - } - catch (JsonMappingException e) { - Assert.assertTrue(e.getCause() instanceof IllegalArgumentException); - } + private void assertThatDeserializationFailsWithMessage(String json, String message) + { + JsonMappingException e = Assert.assertThrows( + JsonMappingException.class, + () -> mapper.readValue( + mapper.writeValueAsString( + mapper.readValue(json, CoordinatorDynamicConfig.class) + ), + CoordinatorDynamicConfig.class + ) + ); + Assert.assertTrue(e.getCause() instanceof IllegalArgumentException); + IllegalArgumentException cause = (IllegalArgumentException) e.getCause(); + Assert.assertEquals(message, cause.getMessage()); } @Test @@ -820,7 +765,7 @@ public class CoordinatorDynamicConfigTest { Assert.assertEquals( expectedLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, - config.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() + config.getMarkSegmentAsUnusedDelayMillis() ); Assert.assertEquals(expectedMergeBytesLimit, config.getMergeBytesLimit()); Assert.assertEquals(expectedMergeSegmentsLimit, config.getMergeSegmentsLimit()); diff --git a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java index e98aaa0eadd..69200382f35 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java @@ -1449,7 +1449,7 @@ public class DataSourcesResourceTest EasyMock.expect(segmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(EasyMock.eq("datasource1"), EasyMock.anyObject(Interval.class), EasyMock.anyBoolean())) .andReturn(Optional.of(segments)).once(); DruidCoordinator druidCoordinator = EasyMock.createMock(DruidCoordinator.class); - EasyMock.expect(druidCoordinator.computeUnderReplicationCountsPerDataSourcePerTierForSegments(segments)) + EasyMock.expect(druidCoordinator.getTierToDatasourceToUnderReplicatedCount(segments, false)) .andReturn(underReplicationCountsPerDataSourcePerTier).once(); EasyMock.replay(segmentsMetadataManager, druidCoordinator); @@ -1506,7 +1506,7 @@ public class DataSourcesResourceTest EasyMock.expect(segmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(EasyMock.eq("datasource1"), EasyMock.anyObject(Interval.class), EasyMock.anyBoolean())) .andReturn(Optional.of(segments)).once(); DruidCoordinator druidCoordinator = EasyMock.createMock(DruidCoordinator.class); - EasyMock.expect(druidCoordinator.computeUnderReplicationCountsPerDataSourcePerTierForSegmentsUsingClusterView(segments)) + EasyMock.expect(druidCoordinator.getTierToDatasourceToUnderReplicatedCount(segments, true)) .andReturn(underReplicationCountsPerDataSourcePerTier).once(); EasyMock.replay(segmentsMetadataManager, druidCoordinator); diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java index f87b65d606d..f0303451797 100644 --- a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -23,11 +23,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; @@ -37,136 +39,81 @@ import org.apache.druid.timeline.SegmentStatusInCluster; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.Mockito; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; -import java.util.ArrayList; import java.util.List; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; - public class MetadataResourceTest { private static final String DATASOURCE1 = "datasource1"; - private static final String DATASOURCE2 = "datasource2"; private MetadataResource metadataResource; - - private SegmentsMetadataManager segmentsMetadataManager; - private DruidCoordinator coordinator; private HttpServletRequest request; - private final DataSegment dataSegment1 = new DataSegment( - DATASOURCE1, - Intervals.of("2010-01-01/P1D"), - "v0", - null, - null, - null, - null, - 0x9, - 10 - ); - - private final DataSegment dataSegment2 = new DataSegment( - DATASOURCE1, - Intervals.of("2010-01-22/P1D"), - "v0", - null, - null, - null, - null, - 0x9, - 20 - ); - - private final DataSegment dataSegment3 = new DataSegment( - DATASOURCE2, - Intervals.of("2010-01-01/P1M"), - "v0", - null, - null, - null, - null, - 0x9, - 30 - ); - - private final DataSegment dataSegment4 = new DataSegment( - DATASOURCE2, - Intervals.of("2010-01-02/P1D"), - "v0", - null, - null, - null, - null, - 0x9, - 35 - ); - + private final DataSegment[] segments = + CreateDataSegments.ofDatasource(DATASOURCE1) + .forIntervals(2, Granularities.DAY) + .withNumPartitions(2) + .eachOfSizeInMb(500) + .toArray(new DataSegment[0]); + @Before public void setUp() { - request = mock(HttpServletRequest.class); - doReturn(mock(AuthenticationResult.class)).when(request).getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT); + request = Mockito.mock(HttpServletRequest.class); + Mockito.doReturn(Mockito.mock(AuthenticationResult.class)) + .when(request).getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT); - segmentsMetadataManager = mock(SegmentsMetadataManager.class); + SegmentsMetadataManager segmentsMetadataManager = Mockito.mock(SegmentsMetadataManager.class); ImmutableDruidDataSource druidDataSource1 = new ImmutableDruidDataSource( DATASOURCE1, ImmutableMap.of(), - ImmutableList.of( - dataSegment1, - dataSegment2 - ) + ImmutableList.of(segments[0], segments[1], segments[2], segments[3]) ); - ImmutableDruidDataSource druidDataSource2 = new ImmutableDruidDataSource( - DATASOURCE1, - ImmutableMap.of(), - ImmutableList.of( - dataSegment3, - dataSegment4 - ) + DataSourcesSnapshot dataSourcesSnapshot = Mockito.mock(DataSourcesSnapshot.class); + Mockito.doReturn(dataSourcesSnapshot) + .when(segmentsMetadataManager).getSnapshotOfDataSourcesWithAllUsedSegments(); + Mockito.doReturn(ImmutableList.of(druidDataSource1)) + .when(dataSourcesSnapshot).getDataSourcesWithAllUsedSegments(); + + DruidCoordinator coordinator = Mockito.mock(DruidCoordinator.class); + Mockito.doReturn(2).when(coordinator).getReplicationFactor(segments[0].getId()); + Mockito.doReturn(null).when(coordinator).getReplicationFactor(segments[1].getId()); + Mockito.doReturn(1).when(coordinator).getReplicationFactor(segments[2].getId()); + Mockito.doReturn(1).when(coordinator).getReplicationFactor(segments[3].getId()); + Mockito.doReturn(ImmutableSet.of(segments[3])) + .when(dataSourcesSnapshot).getOvershadowedSegments(); + + metadataResource = new MetadataResource( + segmentsMetadataManager, + Mockito.mock(IndexerMetadataStorageCoordinator.class), + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + coordinator, + new ObjectMapper() ); - - DataSourcesSnapshot dataSourcesSnapshot = mock(DataSourcesSnapshot.class); - doReturn(dataSourcesSnapshot).when(segmentsMetadataManager).getSnapshotOfDataSourcesWithAllUsedSegments(); - doReturn(ImmutableList.of(druidDataSource1, druidDataSource2)).when(dataSourcesSnapshot).getDataSourcesWithAllUsedSegments(); - - coordinator = mock(DruidCoordinator.class); - doReturn(2).when(coordinator).getReplicationFactorForSegment(dataSegment1.getId()); - doReturn(null).when(coordinator).getReplicationFactorForSegment(dataSegment2.getId()); - doReturn(1).when(coordinator).getReplicationFactorForSegment(dataSegment3.getId()); - doReturn(1).when(coordinator).getReplicationFactorForSegment(dataSegment4.getId()); - doReturn(ImmutableSet.of(dataSegment4)).when(dataSourcesSnapshot).getOvershadowedSegments(); - - metadataResource = new MetadataResource(segmentsMetadataManager, mock(IndexerMetadataStorageCoordinator.class), AuthTestUtils.TEST_AUTHORIZER_MAPPER, coordinator, new ObjectMapper()); } @Test public void testGetAllSegmentsWithOvershadowedStatus() { - Response response = metadataResource.getAllUsedSegments( - request, - null, - "includeOvershadowedStatus" - ); + Response response = metadataResource.getAllUsedSegments(request, null, "includeOvershadowedStatus"); - List resultList = materializeResponse(response); + final List resultList = extractSegmentStatusList(response); Assert.assertEquals(resultList.size(), 4); - Assert.assertEquals(new SegmentStatusInCluster(dataSegment1, false, 2), resultList.get(0)); - Assert.assertEquals(new SegmentStatusInCluster(dataSegment2, false, null), resultList.get(1)); - Assert.assertEquals(new SegmentStatusInCluster(dataSegment3, false, 1), resultList.get(2)); + Assert.assertEquals(new SegmentStatusInCluster(segments[0], false, 2), resultList.get(0)); + Assert.assertEquals(new SegmentStatusInCluster(segments[1], false, null), resultList.get(1)); + Assert.assertEquals(new SegmentStatusInCluster(segments[2], false, 1), resultList.get(2)); // Replication factor should be 0 as the segment is overshadowed - Assert.assertEquals(new SegmentStatusInCluster(dataSegment4, true, 0), resultList.get(3)); + Assert.assertEquals(new SegmentStatusInCluster(segments[3], true, 0), resultList.get(3)); } - private List materializeResponse(Response response) + private List extractSegmentStatusList(Response response) { - Iterable resultIterator = (Iterable) response.getEntity(); - List segmentStatusInClusters = new ArrayList<>(); - resultIterator.forEach(segmentStatusInClusters::add); - return segmentStatusInClusters; + return Lists.newArrayList( + (Iterable) response.getEntity() + ); } } diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 7e40a816de6..2810359f490 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -76,12 +76,11 @@ import org.apache.druid.metadata.SegmentsMetadataManagerProvider; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.server.audit.AuditManagerProvider; -import org.apache.druid.server.coordinator.BalancerStrategyFactory; -import org.apache.druid.server.coordinator.CachingCostBalancerStrategyConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.KillStalePendingSegments; -import org.apache.druid.server.coordinator.LoadQueueTaskMaster; +import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; +import org.apache.druid.server.coordinator.balancer.CachingCostBalancerStrategyConfig; import org.apache.druid.server.coordinator.duty.CompactionSegmentSearchPolicy; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDuty; import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroup; @@ -94,6 +93,7 @@ import org.apache.druid.server.coordinator.duty.KillRules; import org.apache.druid.server.coordinator.duty.KillSupervisors; import org.apache.druid.server.coordinator.duty.KillUnusedSegments; import org.apache.druid.server.coordinator.duty.NewestSegmentFirstPolicy; +import org.apache.druid.server.coordinator.loading.LoadQueueTaskMaster; import org.apache.druid.server.http.ClusterResource; import org.apache.druid.server.http.CompactionResource; import org.apache.druid.server.http.CoordinatorCompactionConfigsResource; From 09d6c5a45ed9c737e711403a864903dcbbf22830 Mon Sep 17 00:00:00 2001 From: Rohan Garg <7731512+rohangarg@users.noreply.github.com> Date: Tue, 20 Jun 2023 04:30:40 +0530 Subject: [PATCH 08/74] Decouple logical planning and native query generation in SQL planning (#14232) Add a new planning strategy that explicitly decouples the DAG from building the native query. With this mode, it is Calcite's job to generate a "logical DAG" which is all of the various DruidProject, DruidFilter, etc. nodes. We then take those nodes and use them to build a native query. The current commit doesn't pass all tests, but it does work for some things and is a decent starting baseline. --- .../plan/volcano/DruidVolcanoCost.java | 288 ++++++++++++ .../calcite/planner/CalciteRulesManager.java | 65 ++- .../calcite/planner/DruidQueryGenerator.java | 336 ++++++++++++++ .../sql/calcite/planner/PlannerConfig.java | 32 +- .../sql/calcite/planner/PlannerFactory.java | 15 +- .../sql/calcite/planner/QueryHandler.java | 119 +++-- .../druid/sql/calcite/rel/CostEstimates.java | 22 +- .../sql/calcite/rel/PartialDruidQuery.java | 58 ++- .../calcite/rel/logical/DruidAggregate.java | 101 +++++ .../sql/calcite/rel/logical/DruidFilter.java | 54 +++ .../rel/logical/DruidLogicalConvention.java | 93 ++++ .../calcite/rel/logical/DruidLogicalNode.java | 30 ++ .../sql/calcite/rel/logical/DruidProject.java | 101 +++++ .../sql/calcite/rel/logical/DruidSort.java | 97 ++++ .../calcite/rel/logical/DruidTableScan.java | 111 +++++ .../sql/calcite/rel/logical/DruidValues.java | 64 +++ .../calcite/rule/DruidLogicalValuesRule.java | 2 +- .../DruidAggregateCaseToFilterRule.java | 339 ++++++++++++++ .../rule/logical/DruidAggregateRule.java | 88 ++++ .../calcite/rule/logical/DruidFilterRule.java | 62 +++ .../rule/logical/DruidLogicalRules.java | 90 ++++ .../rule/logical/DruidProjectRule.java | 59 +++ .../calcite/rule/logical/DruidSortRule.java | 60 +++ .../rule/logical/DruidTableScanRule.java | 54 +++ .../calcite/rule/logical/DruidValuesRule.java | 58 +++ .../sql/calcite/BaseCalciteQueryTest.java | 12 + .../DecoupledPlanningCalciteQueryTest.java | 425 ++++++++++++++++++ .../druid/sql/calcite/QueryTestBuilder.java | 13 +- 28 files changed, 2778 insertions(+), 70 deletions(-) create mode 100644 sql/src/main/java/org/apache/calcite/plan/volcano/DruidVolcanoCost.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidQueryGenerator.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidAggregate.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidFilter.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidLogicalConvention.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidLogicalNode.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidProject.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidSort.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidTableScan.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidValues.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidAggregateCaseToFilterRule.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidAggregateRule.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidFilterRule.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidLogicalRules.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidProjectRule.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidSortRule.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidTableScanRule.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidValuesRule.java create mode 100644 sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java diff --git a/sql/src/main/java/org/apache/calcite/plan/volcano/DruidVolcanoCost.java b/sql/src/main/java/org/apache/calcite/plan/volcano/DruidVolcanoCost.java new file mode 100644 index 00000000000..c26fe21c692 --- /dev/null +++ b/sql/src/main/java/org/apache/calcite/plan/volcano/DruidVolcanoCost.java @@ -0,0 +1,288 @@ +/* + * 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. + */ + +//CHECKSTYLE.OFF: PackageName - Must be in Calcite + +package org.apache.calcite.plan.volcano; + +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptCostFactory; +import org.apache.calcite.plan.RelOptUtil; + +import java.util.Objects; + +/** + * Druid's extension to {@link VolcanoCost}. The difference between the two is in + * comparing two costs. Druid's cost model gives most weightage to rowCount, then to cpuCost and then lastly ioCost. + */ +public class DruidVolcanoCost implements RelOptCost +{ + + static final DruidVolcanoCost INFINITY = new DruidVolcanoCost( + Double.POSITIVE_INFINITY, + Double.POSITIVE_INFINITY, + Double.POSITIVE_INFINITY + ) + { + @Override + public String toString() + { + return "{inf}"; + } + }; + + //CHECKSTYLE.OFF: Regexp + static final DruidVolcanoCost HUGE = new DruidVolcanoCost(Double.MAX_VALUE, Double.MAX_VALUE, Double.MAX_VALUE) { + @Override + public String toString() + { + return "{huge}"; + } + }; + //CHECKSTYLE.ON: Regexp + + static final DruidVolcanoCost ZERO = + new DruidVolcanoCost(0.0, 0.0, 0.0) + { + @Override + public String toString() + { + return "{0}"; + } + }; + + static final DruidVolcanoCost TINY = + new DruidVolcanoCost(1.0, 1.0, 0.0) + { + @Override + public String toString() + { + return "{tiny}"; + } + }; + + public static final RelOptCostFactory FACTORY = new Factory(); + + final double cpu; + final double io; + final double rowCount; + + DruidVolcanoCost(double rowCount, double cpu, double io) + { + this.rowCount = rowCount; + this.cpu = cpu; + this.io = io; + } + + @Override + public double getCpu() + { + return cpu; + } + + @Override + public boolean isInfinite() + { + return (this == INFINITY) + || (this.rowCount == Double.POSITIVE_INFINITY) + || (this.cpu == Double.POSITIVE_INFINITY) + || (this.io == Double.POSITIVE_INFINITY); + } + + @Override + public double getIo() + { + return io; + } + + @Override + public boolean isLe(RelOptCost other) + { + DruidVolcanoCost that = (DruidVolcanoCost) other; + return (this == that) + || ((this.rowCount < that.rowCount) + || (this.rowCount == that.rowCount && this.cpu < that.cpu) + || (this.rowCount == that.rowCount && this.cpu == that.cpu && this.io <= that.io)); + } + + @Override + public boolean isLt(RelOptCost other) + { + return isLe(other) && !equals(other); + } + + @Override + public double getRows() + { + return rowCount; + } + + @Override + public int hashCode() + { + return Objects.hash(rowCount, cpu, io); + } + + @Override + public boolean equals(RelOptCost other) + { + return this == other + || other instanceof DruidVolcanoCost + && (this.rowCount == ((DruidVolcanoCost) other).rowCount) + && (this.cpu == ((DruidVolcanoCost) other).cpu) + && (this.io == ((DruidVolcanoCost) other).io); + } + + @Override + public boolean equals(Object obj) + { + if (obj instanceof DruidVolcanoCost) { + return equals((DruidVolcanoCost) obj); + } + return false; + } + + @Override + public boolean isEqWithEpsilon(RelOptCost other) + { + if (!(other instanceof DruidVolcanoCost)) { + return false; + } + DruidVolcanoCost that = (DruidVolcanoCost) other; + return (this == that) + || ((Math.abs(this.rowCount - that.rowCount) < RelOptUtil.EPSILON) + && (Math.abs(this.cpu - that.cpu) < RelOptUtil.EPSILON) + && (Math.abs(this.io - that.io) < RelOptUtil.EPSILON)); + } + + @Override + public RelOptCost minus(RelOptCost other) + { + if (this == INFINITY) { + return this; + } + DruidVolcanoCost that = (DruidVolcanoCost) other; + return new DruidVolcanoCost( + this.rowCount - that.rowCount, + this.cpu - that.cpu, + this.io - that.io + ); + } + + @Override + public RelOptCost multiplyBy(double factor) + { + if (this == INFINITY) { + return this; + } + return new DruidVolcanoCost(rowCount * factor, cpu * factor, io * factor); + } + + @Override + public double divideBy(RelOptCost cost) + { + // Compute the geometric average of the ratios of all of the factors + // which are non-zero and finite. + DruidVolcanoCost that = (DruidVolcanoCost) cost; + double d = 1; + double n = 0; + if ((this.rowCount != 0) + && !Double.isInfinite(this.rowCount) + && (that.rowCount != 0) + && !Double.isInfinite(that.rowCount)) { + d *= this.rowCount / that.rowCount; + ++n; + } + if ((this.cpu != 0) + && !Double.isInfinite(this.cpu) + && (that.cpu != 0) + && !Double.isInfinite(that.cpu)) { + d *= this.cpu / that.cpu; + ++n; + } + if ((this.io != 0) + && !Double.isInfinite(this.io) + && (that.io != 0) + && !Double.isInfinite(that.io)) { + d *= this.io / that.io; + ++n; + } + if (n == 0) { + return 1.0; + } + return Math.pow(d, 1 / n); + } + + @Override + public RelOptCost plus(RelOptCost other) + { + DruidVolcanoCost that = (DruidVolcanoCost) other; + if ((this == INFINITY) || (that == INFINITY)) { + return INFINITY; + } + return new DruidVolcanoCost( + this.rowCount + that.rowCount, + this.cpu + that.cpu, + this.io + that.io + ); + } + + @Override + public String toString() + { + return "{" + rowCount + " rows, " + cpu + " cpu, " + io + " io}"; + } + + /** + * Implementation of {@link RelOptCostFactory} + * that creates {@link DruidVolcanoCost}s. + */ + public static class Factory implements RelOptCostFactory + { + @Override + public RelOptCost makeCost(double dRows, double dCpu, double dIo) + { + return new DruidVolcanoCost(dRows, dCpu, dIo); + } + + @Override + public RelOptCost makeHugeCost() + { + return DruidVolcanoCost.HUGE; + } + + @Override + public RelOptCost makeInfiniteCost() + { + return DruidVolcanoCost.INFINITY; + } + + @Override + public RelOptCost makeTinyCost() + { + return DruidVolcanoCost.TINY; + } + + @Override + public RelOptCost makeZeroCost() + { + return DruidVolcanoCost.ZERO; + } + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java index 5cc56f7e4c4..5bf4bee733c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java @@ -26,6 +26,7 @@ import org.apache.calcite.plan.RelOptLattice; import org.apache.calcite.plan.RelOptMaterialization; import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.plan.hep.HepProgram; import org.apache.calcite.plan.hep.HepProgramBuilder; @@ -74,11 +75,14 @@ import org.apache.calcite.rel.rules.TableScanRule; import org.apache.calcite.rel.rules.UnionPullUpConstantsRule; import org.apache.calcite.rel.rules.UnionToDistinctRule; import org.apache.calcite.rel.rules.ValuesReduceRule; +import org.apache.calcite.sql.SqlExplainFormat; +import org.apache.calcite.sql.SqlExplainLevel; import org.apache.calcite.sql2rel.RelDecorrelator; import org.apache.calcite.sql2rel.RelFieldTrimmer; import org.apache.calcite.tools.Program; import org.apache.calcite.tools.Programs; import org.apache.calcite.tools.RelBuilder; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.sql.calcite.external.ExternalTableScanRule; import org.apache.druid.sql.calcite.rule.DruidLogicalValuesRule; import org.apache.druid.sql.calcite.rule.DruidRelToDruidRule; @@ -88,6 +92,8 @@ import org.apache.druid.sql.calcite.rule.ExtensionCalciteRuleProvider; import org.apache.druid.sql.calcite.rule.FilterJoinExcludePushToChildRule; import org.apache.druid.sql.calcite.rule.ProjectAggregatePruneUnusedCallRule; import org.apache.druid.sql.calcite.rule.SortCollapseRule; +import org.apache.druid.sql.calcite.rule.logical.DruidAggregateCaseToFilterRule; +import org.apache.druid.sql.calcite.rule.logical.DruidLogicalRules; import org.apache.druid.sql.calcite.run.EngineFeature; import java.util.List; @@ -95,8 +101,11 @@ import java.util.Set; public class CalciteRulesManager { + private static final Logger log = new Logger(CalciteRulesManager.class); + public static final int DRUID_CONVENTION_RULES = 0; public static final int BINDABLE_CONVENTION_RULES = 1; + public static final int DRUID_DAG_CONVENTION_RULES = 2; // Due to Calcite bug (CALCITE-3845), ReduceExpressionsRule can considered expression which is the same as the // previous input expression as reduced. Basically, the expression is actually not reduced but is still considered as @@ -249,12 +258,56 @@ public class CalciteRulesManager buildHepProgram(REDUCTION_RULES, true, DefaultRelMetadataProvider.INSTANCE, HEP_DEFAULT_MATCH_LIMIT) ); + boolean isDebug = plannerContext.queryContext().isDebug(); return ImmutableList.of( Programs.sequence(preProgram, Programs.ofRules(druidConventionRuleSet(plannerContext))), - Programs.sequence(preProgram, Programs.ofRules(bindableConventionRuleSet(plannerContext))) + Programs.sequence(preProgram, Programs.ofRules(bindableConventionRuleSet(plannerContext))), + Programs.sequence( + // currently, adding logging program after every stage for easier debugging + new LoggingProgram("Start", isDebug), + Programs.subQuery(DefaultRelMetadataProvider.INSTANCE), + new LoggingProgram("After subquery program", isDebug), + DecorrelateAndTrimFieldsProgram.INSTANCE, + new LoggingProgram("After trim fields and decorelate program", isDebug), + buildHepProgram(REDUCTION_RULES, true, DefaultRelMetadataProvider.INSTANCE, HEP_DEFAULT_MATCH_LIMIT), + new LoggingProgram("After hep planner program", isDebug), + Programs.ofRules(logicalConventionRuleSet(plannerContext)), + new LoggingProgram("After volcano planner program", isDebug) + ) ); } + private static class LoggingProgram implements Program + { + private final String stage; + private final boolean isDebug; + + public LoggingProgram(String stage, boolean isDebug) + { + this.stage = stage; + this.isDebug = isDebug; + } + + @Override + public RelNode run( + RelOptPlanner planner, + RelNode rel, + RelTraitSet requiredOutputTraits, + List materializations, + List lattices + ) + { + if (isDebug) { + log.info( + "%s%n%s", + stage, + RelOptUtil.dumpPlan("", rel, SqlExplainFormat.TEXT, SqlExplainLevel.ALL_ATTRIBUTES) + ); + } + return rel; + } + } + public Program buildHepProgram( final Iterable rules, final boolean noDag, @@ -287,6 +340,16 @@ public class CalciteRulesManager return retVal.build(); } + public List logicalConventionRuleSet(final PlannerContext plannerContext) + { + final ImmutableList.Builder retVal = ImmutableList + .builder() + .addAll(baseRuleSet(plannerContext)) + .add(DruidAggregateCaseToFilterRule.INSTANCE) + .add(new DruidLogicalRules(plannerContext).rules().toArray(new RelOptRule[0])); + return retVal.build(); + } + public List bindableConventionRuleSet(final PlannerContext plannerContext) { return ImmutableList.builder() diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidQueryGenerator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidQueryGenerator.java new file mode 100644 index 00000000000..87d46809cd5 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidQueryGenerator.java @@ -0,0 +1,336 @@ +/* + * 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.druid.sql.calcite.planner; + +import com.google.common.collect.ImmutableList; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelShuttleImpl; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.core.TableFunctionScan; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.calcite.rel.logical.LogicalCorrelate; +import org.apache.calcite.rel.logical.LogicalExchange; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.calcite.rel.logical.LogicalIntersect; +import org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.calcite.rel.logical.LogicalMatch; +import org.apache.calcite.rel.logical.LogicalMinus; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.calcite.rel.logical.LogicalSort; +import org.apache.calcite.rel.logical.LogicalUnion; +import org.apache.calcite.rel.logical.LogicalValues; +import org.apache.calcite.rex.RexLiteral; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.rel.PartialDruidQuery; +import org.apache.druid.sql.calcite.rel.logical.DruidTableScan; +import org.apache.druid.sql.calcite.rule.DruidLogicalValuesRule; +import org.apache.druid.sql.calcite.table.DruidTable; +import org.apache.druid.sql.calcite.table.InlineTable; +import org.apache.druid.sql.calcite.table.RowSignatures; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + + +/** + * Converts a DAG of {@link org.apache.druid.sql.calcite.rel.logical.DruidLogicalNode} convention to a native + * Druid query for execution. The convertion is done via a {@link org.apache.calcite.rel.RelShuttle} visitor + * implementation. + */ +public class DruidQueryGenerator extends RelShuttleImpl +{ + private final List queryList = new ArrayList<>(); + private final List queryTables = new ArrayList<>(); + private final PlannerContext plannerContext; + private PartialDruidQuery partialDruidQuery; + private PartialDruidQuery.Stage currentStage = null; + private DruidTable currentTable = null; + private boolean isRoot = true; + + public DruidQueryGenerator(PlannerContext plannerContext) + { + this.plannerContext = plannerContext; + } + + @Override + public RelNode visit(TableScan scan) + { + if (!(scan instanceof DruidTableScan)) { + throw new ISE("Planning hasn't converted logical table scan to druid convention"); + } + DruidTableScan druidTableScan = (DruidTableScan) scan; + isRoot = false; + RelNode result = super.visit(scan); + partialDruidQuery = PartialDruidQuery.create(scan); + currentStage = PartialDruidQuery.Stage.SCAN; + final RelOptTable table = scan.getTable(); + final DruidTable druidTable = table.unwrap(DruidTable.class); + if (druidTable != null) { + currentTable = druidTable; + } + if (druidTableScan.getProject() != null) { + partialDruidQuery = partialDruidQuery.withSelectProject(druidTableScan.getProject()); + currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; + } + return result; + } + + @Override + public RelNode visit(TableFunctionScan scan) + { + return null; + } + + @Override + public RelNode visit(LogicalValues values) + { + isRoot = false; + RelNode result = super.visit(values); + final List> tuples = values.getTuples(); + final List objectTuples = tuples + .stream() + .map(tuple -> tuple + .stream() + .map(v -> DruidLogicalValuesRule.getValueFromLiteral(v, plannerContext)) + .collect(Collectors.toList()) + .toArray(new Object[0]) + ) + .collect(Collectors.toList()); + RowSignature rowSignature = RowSignatures.fromRelDataType( + values.getRowType().getFieldNames(), + values.getRowType() + ); + currentTable = new InlineTable(InlineDataSource.fromIterable(objectTuples, rowSignature)); + if (currentStage == null) { + partialDruidQuery = PartialDruidQuery.create(values); + currentStage = PartialDruidQuery.Stage.SCAN; + } else { + throw new ISE("Values node found at non leaf node in the plan"); + } + return result; + } + + @Override + public RelNode visit(LogicalFilter filter) + { + return visitFilter(filter); + } + + public RelNode visitFilter(Filter filter) + { + isRoot = false; + RelNode result = super.visit(filter); + if (currentStage == PartialDruidQuery.Stage.AGGREGATE) { + partialDruidQuery = partialDruidQuery.withHavingFilter(filter); + currentStage = PartialDruidQuery.Stage.HAVING_FILTER; + } else if (currentStage == PartialDruidQuery.Stage.SCAN) { + partialDruidQuery = partialDruidQuery.withWhereFilter(filter); + currentStage = PartialDruidQuery.Stage.WHERE_FILTER; + } else if (currentStage == PartialDruidQuery.Stage.SELECT_PROJECT) { + PartialDruidQuery old = partialDruidQuery; + partialDruidQuery = PartialDruidQuery.create(old.getScan()); + partialDruidQuery = partialDruidQuery.withWhereFilter(filter); + partialDruidQuery = partialDruidQuery.withSelectProject(old.getSelectProject()); + currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; + } else { + queryList.add(partialDruidQuery); + queryTables.add(currentTable); + partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withWhereFilter(filter); + currentStage = PartialDruidQuery.Stage.WHERE_FILTER; + } + return result; + } + + @Override + public RelNode visit(LogicalProject project) + { + return visitProject(project); + } + + @Override + public RelNode visit(LogicalJoin join) + { + throw new UnsupportedOperationException("Found join"); + } + + @Override + public RelNode visit(LogicalCorrelate correlate) + { + return null; + } + + @Override + public RelNode visit(LogicalUnion union) + { + throw new UnsupportedOperationException("Found union"); + } + + @Override + public RelNode visit(LogicalIntersect intersect) + { + return null; + } + + @Override + public RelNode visit(LogicalMinus minus) + { + return null; + } + + @Override + public RelNode visit(LogicalAggregate aggregate) + { + isRoot = false; + RelNode result = super.visit(aggregate); + if (PartialDruidQuery.Stage.AGGREGATE.canFollow(currentStage)) { + partialDruidQuery = partialDruidQuery.withAggregate(aggregate); + } else { + queryList.add(partialDruidQuery); + queryTables.add(currentTable); + partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withAggregate(aggregate); + } + currentStage = PartialDruidQuery.Stage.AGGREGATE; + return result; + } + + @Override + public RelNode visit(LogicalMatch match) + { + return null; + } + + @Override + public RelNode visit(LogicalSort sort) + { + return visitSort(sort); + } + + @Override + public RelNode visit(LogicalExchange exchange) + { + return null; + } + + private RelNode visitProject(Project project) + { + boolean rootForReal = isRoot; + isRoot = false; + RelNode result = super.visit(project); + if (rootForReal && (currentStage == PartialDruidQuery.Stage.AGGREGATE + || currentStage == PartialDruidQuery.Stage.HAVING_FILTER)) { + partialDruidQuery = partialDruidQuery.withAggregateProject(project); + currentStage = PartialDruidQuery.Stage.AGGREGATE_PROJECT; + } else if (currentStage == PartialDruidQuery.Stage.SCAN || currentStage == PartialDruidQuery.Stage.WHERE_FILTER) { + partialDruidQuery = partialDruidQuery.withSelectProject(project); + currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; + } else if (currentStage == PartialDruidQuery.Stage.SELECT_PROJECT) { + partialDruidQuery = partialDruidQuery.mergeProject(project); + currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; + } else if (currentStage == PartialDruidQuery.Stage.SORT) { + partialDruidQuery = partialDruidQuery.withSortProject(project); + currentStage = PartialDruidQuery.Stage.SORT_PROJECT; + } else { + queryList.add(partialDruidQuery); + queryTables.add(currentTable); + partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withSelectProject(project); + currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; + } + return result; + } + + private RelNode visitSort(Sort sort) + { + isRoot = false; + RelNode result = super.visit(sort); + if (PartialDruidQuery.Stage.SORT.canFollow(currentStage)) { + partialDruidQuery = partialDruidQuery.withSort(sort); + } else { + queryList.add(partialDruidQuery); + queryTables.add(currentTable); + partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withSort(sort); + } + currentStage = PartialDruidQuery.Stage.SORT; + return result; + } + + private RelNode visitAggregate(Aggregate aggregate) + { + isRoot = false; + RelNode result = super.visit(aggregate); + if (PartialDruidQuery.Stage.AGGREGATE.canFollow(currentStage)) { + partialDruidQuery = partialDruidQuery.withAggregate(aggregate); + } else { + queryList.add(partialDruidQuery); + queryTables.add(currentTable); + partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withAggregate(aggregate); + } + currentStage = PartialDruidQuery.Stage.AGGREGATE; + return result; + } + + @Override + public RelNode visit(RelNode other) + { + if (other instanceof TableScan) { + return visit((TableScan) other); + } else if (other instanceof Project) { + return visitProject((Project) other); + } else if (other instanceof Sort) { + return visitSort((Sort) other); + } else if (other instanceof Aggregate) { + return visitAggregate((Aggregate) other); + } else if (other instanceof Filter) { + return visitFilter((Filter) other); + } else if (other instanceof LogicalValues) { + return visit((LogicalValues) other); + } + + return super.visit(other); + } + + public PartialDruidQuery getPartialDruidQuery() + { + return partialDruidQuery; + } + + public List getQueryList() + { + return queryList; + } + + public List getQueryTables() + { + return queryTables; + } + + public DruidTable getCurrentTable() + { + return currentTable; + } + +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java index 0a1c6bb68f1..75887bcbec1 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java @@ -36,6 +36,7 @@ public class PlannerConfig public static final String CTX_KEY_USE_NATIVE_QUERY_EXPLAIN = "useNativeQueryExplain"; public static final String CTX_KEY_FORCE_EXPRESSION_VIRTUAL_COLUMNS = "forceExpressionVirtualColumns"; public static final String CTX_MAX_NUMERIC_IN_FILTERS = "maxNumericInFilters"; + public static final String CTX_NATIVE_QUERY_SQL_PLANNING_MODE = "plannerStrategy"; public static final int NUM_FILTER_NOT_USED = -1; @JsonProperty @@ -71,6 +72,11 @@ public class PlannerConfig @JsonProperty private int maxNumericInFilters = NUM_FILTER_NOT_USED; + @JsonProperty + private String nativeQuerySqlPlanningMode = NATIVE_QUERY_SQL_PLANNING_MODE_COUPLED; // can be COUPLED or DECOUPLED + public static final String NATIVE_QUERY_SQL_PLANNING_MODE_COUPLED = "COUPLED"; + public static final String NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED = "DECOUPLED"; + private boolean serializeComplexValues = true; public int getMaxNumericInFilters() @@ -137,6 +143,11 @@ public class PlannerConfig return forceExpressionVirtualColumns; } + public String getNativeQuerySqlPlanningMode() + { + return nativeQuerySqlPlanningMode; + } + public PlannerConfig withOverrides(final Map queryContext) { if (queryContext.isEmpty()) { @@ -168,7 +179,8 @@ public class PlannerConfig useGroupingSetForExactDistinct == that.useGroupingSetForExactDistinct && computeInnerJoinCostAsFilter == that.computeInnerJoinCostAsFilter && authorizeSystemTablesDirectly == that.authorizeSystemTablesDirectly && - maxNumericInFilters == that.maxNumericInFilters; + maxNumericInFilters == that.maxNumericInFilters && + nativeQuerySqlPlanningMode.equals(that.nativeQuerySqlPlanningMode); } @Override @@ -183,7 +195,8 @@ public class PlannerConfig sqlTimeZone, serializeComplexValues, useNativeQueryExplain, - forceExpressionVirtualColumns + forceExpressionVirtualColumns, + nativeQuerySqlPlanningMode ); } @@ -198,6 +211,7 @@ public class PlannerConfig ", sqlTimeZone=" + sqlTimeZone + ", serializeComplexValues=" + serializeComplexValues + ", useNativeQueryExplain=" + useNativeQueryExplain + + ", nativeQuerySqlPlanningMode=" + nativeQuerySqlPlanningMode + '}'; } @@ -231,6 +245,7 @@ public class PlannerConfig private boolean forceExpressionVirtualColumns; private int maxNumericInFilters; private boolean serializeComplexValues; + private String nativeQuerySqlPlanningMode; public Builder(PlannerConfig base) { @@ -249,6 +264,7 @@ public class PlannerConfig forceExpressionVirtualColumns = base.isForceExpressionVirtualColumns(); maxNumericInFilters = base.getMaxNumericInFilters(); serializeComplexValues = base.shouldSerializeComplexValues(); + nativeQuerySqlPlanningMode = base.getNativeQuerySqlPlanningMode(); } public Builder requireTimeCondition(boolean option) @@ -317,6 +333,12 @@ public class PlannerConfig return this; } + public Builder nativeQuerySqlPlanningMode(String mode) + { + this.nativeQuerySqlPlanningMode = mode; + return this; + } + public Builder withOverrides(final Map queryContext) { useApproximateCountDistinct = QueryContexts.parseBoolean( @@ -357,6 +379,11 @@ public class PlannerConfig maxNumericInFilters = validateMaxNumericInFilters( queryContextMaxNumericInFilters, maxNumericInFilters); + nativeQuerySqlPlanningMode = QueryContexts.parseString( + queryContext, + CTX_NATIVE_QUERY_SQL_PLANNING_MODE, + nativeQuerySqlPlanningMode + ); return this; } @@ -397,6 +424,7 @@ public class PlannerConfig config.maxNumericInFilters = maxNumericInFilters; config.forceExpressionVirtualColumns = forceExpressionVirtualColumns; config.serializeComplexValues = serializeComplexValues; + config.nativeQuerySqlPlanningMode = nativeQuerySqlPlanningMode; return config; } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java index 691c33567a8..9780eaa0820 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java @@ -29,6 +29,7 @@ import org.apache.calcite.config.CalciteConnectionConfig; import org.apache.calcite.config.CalciteConnectionConfigImpl; import org.apache.calcite.plan.Context; import org.apache.calcite.plan.ConventionTraitDef; +import org.apache.calcite.plan.volcano.DruidVolcanoCost; import org.apache.calcite.rel.RelCollationTraitDef; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.validate.SqlConformance; @@ -145,7 +146,7 @@ public class PlannerFactory extends PlannerToolbox plannerContext.queryContext().getInSubQueryThreshold() ) .build(); - return Frameworks + Frameworks.ConfigBuilder frameworkConfigBuilder = Frameworks .newConfigBuilder() .parserConfig(PARSER_CONFIG) .traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE) @@ -184,7 +185,15 @@ public class PlannerFactory extends PlannerToolbox return null; } } - }) - .build(); + }); + + if (PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED + .equals(plannerConfig().getNativeQuerySqlPlanningMode()) + ) { + frameworkConfigBuilder.costFactory(new DruidVolcanoCost.Factory()); + } + + return frameworkConfigBuilder.build(); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index c11d600e262..ac3c6faff18 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -56,10 +56,12 @@ import org.apache.calcite.tools.ValidationException; import org.apache.calcite.util.Pair; import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidSqlInput; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryDataSource; import org.apache.druid.server.QueryResponse; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; @@ -68,6 +70,7 @@ import org.apache.druid.sql.calcite.rel.DruidConvention; import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.sql.calcite.rel.DruidRel; import org.apache.druid.sql.calcite.rel.DruidUnionRel; +import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.run.QueryMaker; import org.apache.druid.sql.calcite.table.DruidTable; @@ -531,42 +534,94 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand ); QueryValidations.validateLogicalQueryForDruid(handlerContext.plannerContext(), parameterized); CalcitePlanner planner = handlerContext.planner(); - final DruidRel druidRel = (DruidRel) planner.transform( - CalciteRulesManager.DRUID_CONVENTION_RULES, - planner.getEmptyTraitSet() - .replace(DruidConvention.instance()) - .plus(rootQueryRel.collation), - parameterized - ); - handlerContext.hook().captureDruidRel(druidRel); - if (explain != null) { - return planExplanation(possiblyLimitedRoot, druidRel, true); - } else { - // Compute row type. - final RelDataType rowType = prepareResult.getReturnedRowType(); - - // Start the query. - final Supplier> resultsSupplier = () -> { - // sanity check - final Set readResourceActions = - plannerContext.getResourceActions() - .stream() - .filter(action -> action.getAction() == Action.READ) - .collect(Collectors.toSet()); - Preconditions.checkState( - readResourceActions.isEmpty() == druidRel.getDataSourceNames().isEmpty() - // The resources found in the plannerContext can be less than the datasources in - // the query plan, because the query planner can eliminate empty tables by replacing - // them with InlineDataSource of empty rows. - || readResourceActions.size() >= druidRel.getDataSourceNames().size(), - "Authorization sanity check failed" + if (plannerContext.getPlannerConfig() + .getNativeQuerySqlPlanningMode() + .equals(PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED) + ) { + RelNode newRoot = parameterized; + newRoot = planner.transform( + CalciteRulesManager.DRUID_DAG_CONVENTION_RULES, + planner.getEmptyTraitSet() + .plus(rootQueryRel.collation) + .plus(DruidLogicalConvention.instance()), + newRoot + ); + DruidQueryGenerator shuttle = new DruidQueryGenerator(plannerContext); + newRoot.accept(shuttle); + log.info("PartialDruidQuery : " + shuttle.getPartialDruidQuery()); + shuttle.getQueryList().add(shuttle.getPartialDruidQuery()); // add topmost query to the list + shuttle.getQueryTables().add(shuttle.getCurrentTable()); + assert !shuttle.getQueryList().isEmpty(); + log.info("query list size " + shuttle.getQueryList().size()); + log.info("query tables size " + shuttle.getQueryTables().size()); + // build bottom-most query + DruidQuery baseQuery = shuttle.getQueryList().get(0).build( + shuttle.getQueryTables().get(0).getDataSource(), + shuttle.getQueryTables().get(0).getRowSignature(), + plannerContext, + rexBuilder, + shuttle.getQueryList().size() != 1, + null + ); + // build outer queries + for (int i = 1; i < shuttle.getQueryList().size(); i++) { + baseQuery = shuttle.getQueryList().get(i).build( + new QueryDataSource(baseQuery.getQuery()), + baseQuery.getOutputRowSignature(), + plannerContext, + rexBuilder, + false ); + } + try { + log.info("final query : " + + new DefaultObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(baseQuery.getQuery())); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + DruidQuery finalBaseQuery = baseQuery; + final Supplier> resultsSupplier = () -> plannerContext.getQueryMaker().runQuery(finalBaseQuery); - return druidRel.runQuery(); - }; + return new PlannerResult(resultsSupplier, finalBaseQuery.getOutputRowType()); + } else { + final DruidRel druidRel = (DruidRel) planner.transform( + CalciteRulesManager.DRUID_CONVENTION_RULES, + planner.getEmptyTraitSet() + .replace(DruidConvention.instance()) + .plus(rootQueryRel.collation), + parameterized + ); + handlerContext.hook().captureDruidRel(druidRel); + if (explain != null) { + return planExplanation(possiblyLimitedRoot, druidRel, true); + } else { + // Compute row type. + final RelDataType rowType = prepareResult.getReturnedRowType(); - return new PlannerResult(resultsSupplier, rowType); + // Start the query. + final Supplier> resultsSupplier = () -> { + // sanity check + final Set readResourceActions = + plannerContext.getResourceActions() + .stream() + .filter(action -> action.getAction() == Action.READ) + .collect(Collectors.toSet()); + Preconditions.checkState( + readResourceActions.isEmpty() == druidRel.getDataSourceNames().isEmpty() + // The resources found in the plannerContext can be less than the datasources in + // the query plan, because the query planner can eliminate empty tables by replacing + // them with InlineDataSource of empty rows. + || readResourceActions.size() >= druidRel.getDataSourceNames().size(), + "Authorization sanity check failed" + ); + + return druidRel.runQuery(); + }; + + return new PlannerResult(resultsSupplier, rowType); + } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/CostEstimates.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/CostEstimates.java index 28f7c21182d..7de0bcc56b5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/CostEstimates.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/CostEstimates.java @@ -32,61 +32,61 @@ public class CostEstimates * Per-row base cost. This represents the cost of walking through every row, but not actually reading anything * from them or computing any aggregations. */ - static final double COST_BASE = 1; + public static final double COST_BASE = 1; /** * Cost to include a column in query output. */ - static final double COST_OUTPUT_COLUMN = 0.05; + public static final double COST_OUTPUT_COLUMN = 0.05; /** * Cost to compute and read an expression. */ - static final double COST_EXPRESSION = 0.25; + public static final double COST_EXPRESSION = 0.25; /** * Cost to compute an aggregation. */ - static final double COST_AGGREGATION = 0.05; + public static final double COST_AGGREGATION = 0.05; /** * Cost per GROUP BY dimension. */ - static final double COST_DIMENSION = 0.25; + public static final double COST_DIMENSION = 0.25; /** * Multiplier to apply when there is a WHERE filter. Encourages pushing down filters and limits through joins and * subqueries when possible. */ - static final double MULTIPLIER_FILTER = 0.1; + public static final double MULTIPLIER_FILTER = 0.1; /** * Multiplier to apply when there is an ORDER BY. Encourages avoiding them when possible. */ - static final double MULTIPLIER_ORDER_BY = 10; + public static final double MULTIPLIER_ORDER_BY = 10; /** * Multiplier to apply when there is a LIMIT. Encourages pushing down limits when possible. */ - static final double MULTIPLIER_LIMIT = 0.5; + public static final double MULTIPLIER_LIMIT = 0.5; /** * Multiplier to apply to an outer query via {@link DruidOuterQueryRel}. Encourages pushing down time-saving * operations to the lowest level of the query stack, because they'll have bigger impact there. */ - static final double MULTIPLIER_OUTER_QUERY = .1; + public static final double MULTIPLIER_OUTER_QUERY = .1; /** * Cost to add to a subquery. Strongly encourages avoiding subqueries, since they must be inlined and then the join * must run on the Broker. */ - static final double COST_SUBQUERY = 1e5; + public static final double COST_SUBQUERY = 1e5; /** * Cost to perform a cross join. Strongly encourages pushing down filters into join conditions, even if it means * we need to add a subquery (this is higher than {@link #COST_SUBQUERY}). */ - static final double COST_JOIN_CROSS = 1e8; + public static final double COST_JOIN_CROSS = 1e8; private CostEstimates() { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java index 068ff49308d..f5d3e5ac8e1 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java @@ -260,24 +260,7 @@ public class PartialDruidQuery if (selectProject == null) { theProject = newSelectProject; } else { - final List newProjectRexNodes = RelOptUtil.pushPastProject( - newSelectProject.getProjects(), - selectProject - ); - - if (RexUtil.isIdentity(newProjectRexNodes, selectProject.getInput().getRowType())) { - // The projection is gone. - theProject = null; - } else { - final RelBuilder relBuilder = builderSupplier.get(); - relBuilder.push(selectProject.getInput()); - relBuilder.project( - newProjectRexNodes, - newSelectProject.getRowType().getFieldNames(), - true - ); - theProject = (Project) relBuilder.build(); - } + return mergeProject(newSelectProject); } return new PartialDruidQuery( @@ -295,6 +278,45 @@ public class PartialDruidQuery ); } + public PartialDruidQuery mergeProject(Project newSelectProject) + { + if (stage() != Stage.SELECT_PROJECT) { + throw new ISE("Expected partial query state to be [%s], but found [%s]", Stage.SELECT_PROJECT, stage()); + } + Project theProject; + final List newProjectRexNodes = RelOptUtil.pushPastProject( + newSelectProject.getProjects(), + selectProject + ); + + if (RexUtil.isIdentity(newProjectRexNodes, selectProject.getInput().getRowType())) { + // The projection is gone. + theProject = null; + } else { + final RelBuilder relBuilder = builderSupplier.get(); + relBuilder.push(selectProject.getInput()); + relBuilder.project( + newProjectRexNodes, + newSelectProject.getRowType().getFieldNames(), + true + ); + theProject = (Project) relBuilder.build(); + } + return new PartialDruidQuery( + builderSupplier, + scan, + whereFilter, + theProject, + aggregate, + aggregateProject, + havingFilter, + sort, + sortProject, + window, + windowProject + ); + } + public PartialDruidQuery withAggregate(final Aggregate newAggregate) { validateStage(Stage.AGGREGATE); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidAggregate.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidAggregate.java new file mode 100644 index 00000000000..711ba26ca61 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidAggregate.java @@ -0,0 +1,101 @@ +/* + * 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.druid.sql.calcite.rel.logical; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelWriter; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.CostEstimates; + +import java.util.List; + +/** + * {@link DruidLogicalNode} convention node for {@link Aggregate} plan node. + */ +public class DruidAggregate extends Aggregate implements DruidLogicalNode +{ + private final PlannerContext plannerContext; + + public DruidAggregate( + RelOptCluster cluster, + RelTraitSet traitSet, + RelNode input, + ImmutableBitSet groupSet, + List groupSets, + List aggCalls, + PlannerContext plannerContext + ) + { + super(cluster, traitSet, input, groupSet, groupSets, aggCalls); + assert getConvention() instanceof DruidLogicalConvention; + this.plannerContext = plannerContext; + } + + @Override + public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) + { + double rowCount = mq.getRowCount(this); + double cost = CostEstimates.COST_DIMENSION * getGroupSet().size(); + for (AggregateCall aggregateCall : getAggCallList()) { + if (aggregateCall.hasFilter()) { + cost += CostEstimates.COST_AGGREGATION * CostEstimates.MULTIPLIER_FILTER; + } else { + cost += CostEstimates.COST_AGGREGATION; + } + } + if (!plannerContext.getPlannerConfig().isUseApproximateCountDistinct() && + getAggCallList().stream().anyMatch(AggregateCall::isDistinct)) { + return planner.getCostFactory().makeInfiniteCost(); + } + return planner.getCostFactory().makeCost(rowCount, cost, 0); + } + + @Override + public final Aggregate copy( + RelTraitSet traitSet, + RelNode input, + ImmutableBitSet groupSet, + List groupSets, + List aggCalls + ) + { + return new DruidAggregate(getCluster(), traitSet, input, groupSet, groupSets, aggCalls, plannerContext); + } + + @Override + public RelWriter explainTerms(RelWriter pw) + { + return super.explainTerms(pw).item("druid", "logical"); + } + + @Override + public double estimateRowCount(RelMetadataQuery mq) + { + return mq.getRowCount(this); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidFilter.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidFilter.java new file mode 100644 index 00000000000..00886fe630e --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidFilter.java @@ -0,0 +1,54 @@ +/* + * 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.druid.sql.calcite.rel.logical; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rex.RexNode; + +/** + * {@link DruidLogicalNode} convention node for {@link Filter} plan node. + */ +public class DruidFilter extends Filter implements DruidLogicalNode +{ + + public DruidFilter(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) + { + super(cluster, traits, child, condition); + assert getConvention() instanceof DruidLogicalConvention; + } + + @Override + public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) + { + return new DruidFilter(getCluster(), getTraitSet(), input, condition); + } + + @Override + public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) + { + return planner.getCostFactory().makeCost(mq.getRowCount(this), 0, 0); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidLogicalConvention.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidLogicalConvention.java new file mode 100644 index 00000000000..0ac8b042ceb --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidLogicalConvention.java @@ -0,0 +1,93 @@ +/* + * 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.druid.sql.calcite.rel.logical; + +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.ConventionTraitDef; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTrait; +import org.apache.calcite.plan.RelTraitDef; +import org.apache.calcite.plan.RelTraitSet; + +/** + * A Calcite convention to produce {@link DruidLogicalNode} based DAG. + */ +public class DruidLogicalConvention implements Convention +{ + + private static final DruidLogicalConvention INSTANCE = new DruidLogicalConvention(); + private static final String NAME = "DRUID_LOGICAL"; + + private DruidLogicalConvention() + { + } + + public static DruidLogicalConvention instance() + { + return INSTANCE; + } + + @Override + public Class getInterface() + { + return DruidLogicalNode.class; + } + + @Override + public String getName() + { + return NAME; + } + + @Override + public boolean canConvertConvention(Convention toConvention) + { + return false; + } + + @Override + public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) + { + return false; + } + + @Override + public RelTraitDef getTraitDef() + { + return ConventionTraitDef.INSTANCE; + } + + @Override + public boolean satisfies(RelTrait trait) + { + return trait.equals(this); + } + + @Override + public void register(RelOptPlanner planner) + { + } + + @Override + public String toString() + { + return NAME; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidLogicalNode.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidLogicalNode.java new file mode 100644 index 00000000000..75029eab1a7 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidLogicalNode.java @@ -0,0 +1,30 @@ +/* + * 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.druid.sql.calcite.rel.logical; + +import org.apache.calcite.rel.RelNode; + +/** + * An interface to mark {@link RelNode} as Druid physical nodes. These physical nodes look a lot same as their logical + * counterparts in Calcite, but they do follow a different costing model. + */ +public interface DruidLogicalNode extends RelNode +{ +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidProject.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidProject.java new file mode 100644 index 00000000000..83e437514ab --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidProject.java @@ -0,0 +1,101 @@ +/* + * 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.druid.sql.calcite.rel.logical; + +import com.google.common.collect.ImmutableSet; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelCollationTraitDef; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelWriter; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.metadata.RelMdCollation; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.sql.calcite.rel.CostEstimates; + +import java.util.List; + +/** + * {@link DruidLogicalNode} convention node for {@link Project} plan node. + */ +public class DruidProject extends Project implements DruidLogicalNode +{ + public DruidProject( + RelOptCluster cluster, + RelTraitSet traitSet, + RelNode input, + List projects, + RelDataType rowType + ) + { + super(cluster, traitSet, input, projects, rowType); + assert getConvention() instanceof DruidLogicalConvention; + } + + @Override + public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) + { + double cost = 0; + double rowCount = mq.getRowCount(getInput()); + for (final RexNode rexNode : getProjects()) { + if (rexNode.isA(SqlKind.INPUT_REF)) { + cost += 0; + } + if (rexNode.getType().getSqlTypeName() == SqlTypeName.BOOLEAN || rexNode.isA(SqlKind.CAST)) { + cost += 0; + } else if (!rexNode.isA(ImmutableSet.of(SqlKind.INPUT_REF, SqlKind.LITERAL))) { + cost += CostEstimates.COST_EXPRESSION; + } + } + // adding atleast 1e-6 cost since zero cost is converted to a tiny cost by the planner which is (1 row, 1 cpu, 0 io) + // that becomes a significant cost in some cases. + return planner.getCostFactory().makeCost(0, Math.max(cost * rowCount, 1e-6), 0); + } + + @Override + public Project copy(RelTraitSet traitSet, RelNode input, List projects, RelDataType rowType) + { + return new DruidProject(getCluster(), traitSet, input, exps, rowType); + } + + @Override + public RelWriter explainTerms(RelWriter pw) + { + return super.explainTerms(pw).item("druid", "logical"); + } + + public static DruidProject create(final RelNode input, final List projects, RelDataType rowType) + { + final RelOptCluster cluster = input.getCluster(); + final RelMetadataQuery mq = cluster.getMetadataQuery(); + final RelTraitSet traitSet = + input.getTraitSet().replaceIfs( + RelCollationTraitDef.INSTANCE, + () -> RelMdCollation.project(mq, input, projects) + ); + return new DruidProject(cluster, traitSet, input, projects, rowType); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidSort.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidSort.java new file mode 100644 index 00000000000..4ad6091ad12 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidSort.java @@ -0,0 +1,97 @@ +/* + * 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.druid.sql.calcite.rel.logical; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelCollation; +import org.apache.calcite.rel.RelCollationTraitDef; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelWriter; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rex.RexNode; +import org.apache.druid.sql.calcite.planner.OffsetLimit; +import org.apache.druid.sql.calcite.rel.CostEstimates; + +/** + * {@link DruidLogicalNode} convention node for {@link Sort} plan node. + */ +public class DruidSort extends Sort implements DruidLogicalNode +{ + private DruidSort( + RelOptCluster cluster, + RelTraitSet traits, + RelNode input, + RelCollation collation, + RexNode offset, + RexNode fetch + ) + { + super(cluster, traits, input, collation, offset, fetch); + assert getConvention() instanceof DruidLogicalConvention; + } + + public static DruidSort create(RelNode input, RelCollation collation, RexNode offset, RexNode fetch) + { + RelOptCluster cluster = input.getCluster(); + collation = RelCollationTraitDef.INSTANCE.canonize(collation); + RelTraitSet traitSet = + input.getTraitSet().replace(DruidLogicalConvention.instance()).replace(collation); + return new DruidSort(cluster, traitSet, input, collation, offset, fetch); + } + + @Override + public Sort copy( + RelTraitSet traitSet, + RelNode newInput, + RelCollation newCollation, + RexNode offset, + RexNode fetch + ) + { + return new DruidSort(getCluster(), traitSet, newInput, newCollation, offset, fetch); + } + + @Override + public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) + { + double cost = 0; + double rowCount = mq.getRowCount(this); + + if (fetch != null) { + OffsetLimit offsetLimit = OffsetLimit.fromSort(this); + rowCount = Math.min(rowCount, offsetLimit.getLimit() - offsetLimit.getOffset()); + } + + if (!getCollation().getFieldCollations().isEmpty() && fetch == null) { + cost = rowCount * CostEstimates.MULTIPLIER_ORDER_BY; + } + return planner.getCostFactory().makeCost(rowCount, cost, 0); + } + + @Override + public RelWriter explainTerms(RelWriter pw) + { + return super.explainTerms(pw).item("druid", "logical"); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidTableScan.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidTableScan.java new file mode 100644 index 00000000000..d601aa4d2f7 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidTableScan.java @@ -0,0 +1,111 @@ +/* + * 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.druid.sql.calcite.rel.logical; + +import com.google.common.collect.ImmutableList; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelCollationTraitDef; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelWriter; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.schema.Table; + +import java.util.List; + +/** + * {@link DruidLogicalNode} convention node for {@link TableScan} plan node. + */ +public class DruidTableScan extends TableScan implements DruidLogicalNode +{ + private final Project project; + + public DruidTableScan( + RelOptCluster cluster, + RelTraitSet traitSet, + RelOptTable table, + Project project + ) + { + super(cluster, traitSet, table); + this.project = project; + assert getConvention() instanceof DruidLogicalConvention; + } + + @Override + public RelNode copy(RelTraitSet traitSet, List inputs) + { + return new DruidTableScan(getCluster(), traitSet, table, project); + } + + @Override + public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) + { + return planner.getCostFactory().makeTinyCost(); + } + + @Override + public double estimateRowCount(RelMetadataQuery mq) + { + return 1_000; + } + + @Override + public RelWriter explainTerms(RelWriter pw) + { + if (project != null) { + project.explainTerms(pw); + } + return super.explainTerms(pw).item("druid", "logical"); + } + + @Override + public RelDataType deriveRowType() + { + if (project != null) { + return project.getRowType(); + } + return super.deriveRowType(); + } + + public Project getProject() + { + return project; + } + + public static DruidTableScan create(RelOptCluster cluster, final RelOptTable relOptTable) + { + final Table table = relOptTable.unwrap(Table.class); + final RelTraitSet traitSet = + cluster.traitSet().replaceIfs(RelCollationTraitDef.INSTANCE, () -> { + if (table != null) { + return table.getStatistic().getCollations(); + } + return ImmutableList.of(); + }); + return new DruidTableScan(cluster, traitSet, relOptTable, null); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidValues.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidValues.java new file mode 100644 index 00000000000..d6a8ca98a22 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidValues.java @@ -0,0 +1,64 @@ +/* + * 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.druid.sql.calcite.rel.logical; + +import com.google.common.collect.ImmutableList; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.logical.LogicalValues; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexLiteral; +import org.apache.druid.sql.calcite.rel.CostEstimates; + +import java.util.List; + +/** + * {@link DruidLogicalNode} convention node for {@link LogicalValues} plan node. + */ +public class DruidValues extends LogicalValues implements DruidLogicalNode +{ + + public DruidValues( + RelOptCluster cluster, + RelTraitSet traitSet, + RelDataType rowType, + ImmutableList> tuples + ) + { + super(cluster, traitSet, rowType, tuples); + assert getConvention() instanceof DruidLogicalConvention; + } + + @Override + public RelNode copy(RelTraitSet traitSet, List inputs) + { + return new DruidValues(getCluster(), traitSet, getRowType(), tuples); + } + + @Override + public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) + { + return planner.getCostFactory().makeCost(CostEstimates.COST_BASE, 0, 0); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRule.java index ea71dfd9098..614ffddf566 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRule.java @@ -92,7 +92,7 @@ public class DruidLogicalValuesRule extends RelOptRule */ @Nullable @VisibleForTesting - static Object getValueFromLiteral(RexLiteral literal, PlannerContext plannerContext) + public static Object getValueFromLiteral(RexLiteral literal, PlannerContext plannerContext) { switch (literal.getType().getSqlTypeName()) { case CHAR: diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidAggregateCaseToFilterRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidAggregateCaseToFilterRule.java new file mode 100644 index 00000000000..b620905f139 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidAggregateCaseToFilterRule.java @@ -0,0 +1,339 @@ +/* + * 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.druid.sql.calcite.rule.logical; + +import com.google.common.collect.ImmutableList; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.RelFactories; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlPostfixOperator; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.tools.RelBuilderFactory; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * A copy of {@link org.apache.calcite.rel.rules.AggregateCaseToFilterRule} except that it fixes a bug to eliminate + * left-over projects for converted aggregates to filter-aggregates. The elimination of left-over projects is necessary + * with the new planning since it determines the cost of the plan and hence determines which plan is going to get picked + * as the cheapest one. + * This fix will also be contributed upstream to Calcite project, and we can remove this rule once the fix is a part of + * the Calcite version we use. + */ +public class DruidAggregateCaseToFilterRule extends RelOptRule +{ + public static final DruidAggregateCaseToFilterRule INSTANCE = + new DruidAggregateCaseToFilterRule(RelFactories.LOGICAL_BUILDER, null); + + /** + * Creates an AggregateCaseToFilterRule. + */ + protected DruidAggregateCaseToFilterRule( + RelBuilderFactory relBuilderFactory, + String description + ) + { + super(operand(Aggregate.class, operand(Project.class, any())), + relBuilderFactory, description + ); + } + + @Override + public boolean matches(final RelOptRuleCall call) + { + final Aggregate aggregate = call.rel(0); + final Project project = call.rel(1); + + for (AggregateCall aggregateCall : aggregate.getAggCallList()) { + final int singleArg = soleArgument(aggregateCall); + if (singleArg >= 0 + && isThreeArgCase(project.getProjects().get(singleArg))) { + return true; + } + } + + return false; + } + + @Override + public void onMatch(RelOptRuleCall call) + { + final Aggregate aggregate = call.rel(0); + final Project project = call.rel(1); + final List newCalls = + new ArrayList<>(aggregate.getAggCallList().size()); + List newProjects; + + // TODO : fix grouping columns + Set groupUsedFields = new HashSet<>(); + for (int fieldNumber : aggregate.getGroupSet()) { + groupUsedFields.add(fieldNumber); + } + + List updatedProjects = new ArrayList<>(); + for (int i = 0; i < project.getProjects().size(); i++) { + if (groupUsedFields.contains(i)) { + updatedProjects.add(project.getProjects().get(i)); + } + } + newProjects = updatedProjects; + + for (AggregateCall aggregateCall : aggregate.getAggCallList()) { + AggregateCall newCall = + transform(aggregateCall, project, newProjects); + + // Possibly CAST the new aggregator to an appropriate type. + newCalls.add(newCall); + } + final RelBuilder relBuilder = call.builder() + .push(project.getInput()) + .project(newProjects); + + final RelBuilder.GroupKey groupKey = + relBuilder.groupKey( + aggregate.getGroupSet(), + aggregate.getGroupSets() + ); + + relBuilder.aggregate(groupKey, newCalls) + .convert(aggregate.getRowType(), false); + + call.transformTo(relBuilder.build()); + call.getPlanner().setImportance(aggregate, 0.0); + } + + private AggregateCall transform(AggregateCall aggregateCall, Project project, List newProjects) + { + final int singleArg = soleArgument(aggregateCall); + if (singleArg < 0) { + Set newFields = new HashSet<>(); + for (int fieldNumber : aggregateCall.getArgList()) { + newProjects.add(project.getProjects().get(fieldNumber)); + newFields.add(newProjects.size() - 1); + } + int newFilterArg = -1; + if (aggregateCall.hasFilter()) { + newProjects.add(project.getProjects().get(aggregateCall.filterArg)); + newFilterArg = newProjects.size() - 1; + } + return AggregateCall.create(aggregateCall.getAggregation(), + aggregateCall.isDistinct(), + aggregateCall.isApproximate(), + aggregateCall.ignoreNulls(), + new ArrayList<>(newFields), + newFilterArg, + aggregateCall.getCollation(), + aggregateCall.getType(), + aggregateCall.getName() + ); + } + + final RexNode rexNode = project.getProjects().get(singleArg); + if (!isThreeArgCase(rexNode)) { + newProjects.add(rexNode); + int callArg = newProjects.size() - 1; + int newFilterArg = -1; + if (aggregateCall.hasFilter()) { + newProjects.add(project.getProjects().get(aggregateCall.filterArg)); + newFilterArg = newProjects.size() - 1; + } + return AggregateCall.create(aggregateCall.getAggregation(), + aggregateCall.isDistinct(), + aggregateCall.isApproximate(), + aggregateCall.ignoreNulls(), + ImmutableList.of(callArg), + newFilterArg, + aggregateCall.getCollation(), + aggregateCall.getType(), + aggregateCall.getName() + ); + } + + final RelOptCluster cluster = project.getCluster(); + final RexBuilder rexBuilder = cluster.getRexBuilder(); + final RexCall caseCall = (RexCall) rexNode; + + // If one arg is null and the other is not, reverse them and set "flip", + // which negates the filter. + final boolean flip = RexLiteral.isNullLiteral(caseCall.operands.get(1)) + && !RexLiteral.isNullLiteral(caseCall.operands.get(2)); + final RexNode arg1 = caseCall.operands.get(flip ? 2 : 1); + final RexNode arg2 = caseCall.operands.get(flip ? 1 : 2); + + // Operand 1: Filter + final SqlPostfixOperator op = + flip ? SqlStdOperatorTable.IS_FALSE : SqlStdOperatorTable.IS_TRUE; + final RexNode filterFromCase = + rexBuilder.makeCall(op, caseCall.operands.get(0)); + + // Combine the CASE filter with an honest-to-goodness SQL FILTER, if the + // latter is present. + final RexNode filter; + if (aggregateCall.filterArg >= 0) { + filter = rexBuilder.makeCall(SqlStdOperatorTable.AND, + project.getProjects().get(aggregateCall.filterArg), filterFromCase + ); + } else { + filter = filterFromCase; + } + + final SqlKind kind = aggregateCall.getAggregation().getKind(); + if (aggregateCall.isDistinct()) { + // Just one style supported: + // COUNT(DISTINCT CASE WHEN x = 'foo' THEN y END) + // => + // COUNT(DISTINCT y) FILTER(WHERE x = 'foo') + + if (kind == SqlKind.COUNT + && RexLiteral.isNullLiteral(arg2)) { + newProjects.add(arg1); + newProjects.add(filter); + return AggregateCall.create(SqlStdOperatorTable.COUNT, true, false, + false, ImmutableList.of(newProjects.size() - 2), + newProjects.size() - 1, RelCollations.EMPTY, + aggregateCall.getType(), aggregateCall.getName() + ); + } + newProjects.add(rexNode); + int callArg = newProjects.size() - 1; + int newFilterArg = -1; + if (aggregateCall.hasFilter()) { + newProjects.add(project.getProjects().get(aggregateCall.filterArg)); + newFilterArg = newProjects.size() - 1; + } + return AggregateCall.create(aggregateCall.getAggregation(), + aggregateCall.isDistinct(), + aggregateCall.isApproximate(), + aggregateCall.ignoreNulls(), + ImmutableList.of(callArg), + newFilterArg, + aggregateCall.getCollation(), + aggregateCall.getType(), + aggregateCall.getName() + ); + } + + // Four styles supported: + // + // A1: AGG(CASE WHEN x = 'foo' THEN cnt END) + // => operands (x = 'foo', cnt, null) + // A2: SUM(CASE WHEN x = 'foo' THEN cnt ELSE 0 END) + // => operands (x = 'foo', cnt, 0); must be SUM + // B: SUM(CASE WHEN x = 'foo' THEN 1 ELSE 0 END) + // => operands (x = 'foo', 1, 0); must be SUM + // C: COUNT(CASE WHEN x = 'foo' THEN 'dummy' END) + // => operands (x = 'foo', 'dummy', null) + + if (kind == SqlKind.COUNT // Case C + && arg1.isA(SqlKind.LITERAL) + && !RexLiteral.isNullLiteral(arg1) + && RexLiteral.isNullLiteral(arg2)) { + newProjects.add(filter); + return AggregateCall.create(SqlStdOperatorTable.COUNT, false, false, + false, ImmutableList.of(), newProjects.size() - 1, + RelCollations.EMPTY, aggregateCall.getType(), + aggregateCall.getName() + ); + } else if (kind == SqlKind.SUM // Case B + && isIntLiteral(arg1) && RexLiteral.intValue(arg1) == 1 + && isIntLiteral(arg2) && RexLiteral.intValue(arg2) == 0) { + + newProjects.add(filter); + final RelDataTypeFactory typeFactory = cluster.getTypeFactory(); + final RelDataType dataType = + typeFactory.createTypeWithNullability( + typeFactory.createSqlType(SqlTypeName.BIGINT), false); + return AggregateCall.create(SqlStdOperatorTable.COUNT, false, false, + false, ImmutableList.of(), newProjects.size() - 1, + RelCollations.EMPTY, dataType, aggregateCall.getName() + ); + } else if ((RexLiteral.isNullLiteral(arg2) // Case A1 + && aggregateCall.getAggregation().allowsFilter()) + || (kind == SqlKind.SUM // Case A2 + && isIntLiteral(arg2) + && RexLiteral.intValue(arg2) == 0)) { + newProjects.add(arg1); + newProjects.add(filter); + return AggregateCall.create(aggregateCall.getAggregation(), false, + false, false, ImmutableList.of(newProjects.size() - 2), + newProjects.size() - 1, RelCollations.EMPTY, + aggregateCall.getType(), aggregateCall.getName() + ); + } else { + newProjects.add(rexNode); + int callArg = newProjects.size() - 1; + int newFilterArg = -1; + if (aggregateCall.hasFilter()) { + newProjects.add(project.getProjects().get(aggregateCall.filterArg)); + newFilterArg = newProjects.size() - 1; + } + return AggregateCall.create(aggregateCall.getAggregation(), + aggregateCall.isDistinct(), + aggregateCall.isApproximate(), + aggregateCall.ignoreNulls(), + ImmutableList.of(callArg), + newFilterArg, + aggregateCall.getCollation(), + aggregateCall.getType(), + aggregateCall.getName() + ); + } + } + + /** + * Returns the argument, if an aggregate call has a single argument, + * otherwise -1. + */ + private static int soleArgument(AggregateCall aggregateCall) + { + return aggregateCall.getArgList().size() == 1 + ? aggregateCall.getArgList().get(0) + : -1; + } + + private static boolean isThreeArgCase(final RexNode rexNode) + { + return rexNode.getKind() == SqlKind.CASE + && ((RexCall) rexNode).operands.size() == 3; + } + + private static boolean isIntLiteral(final RexNode rexNode) + { + return rexNode instanceof RexLiteral + && SqlTypeName.INT_TYPES.contains(rexNode.getType().getSqlTypeName()); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidAggregateRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidAggregateRule.java new file mode 100644 index 00000000000..07ff9cd57b7 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidAggregateRule.java @@ -0,0 +1,88 @@ +/* + * 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.druid.sql.calcite.rule.logical; + +import org.apache.calcite.plan.RelTrait; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelCollation; +import org.apache.calcite.rel.RelCollationTraitDef; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.RelFieldCollation; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.logical.DruidAggregate; +import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; + +import java.util.ArrayList; +import java.util.List; + +/** + * {@link ConverterRule} to convert {@link Aggregate} to {@link DruidAggregate} + */ +public class DruidAggregateRule extends ConverterRule +{ + private final PlannerContext plannerContext; + + public DruidAggregateRule( + Class clazz, + RelTrait in, + RelTrait out, + String descriptionPrefix, + PlannerContext plannerContext + ) + { + super(clazz, in, out, descriptionPrefix); + this.plannerContext = plannerContext; + } + + @Override + public RelNode convert(RelNode rel) + { + LogicalAggregate aggregate = (LogicalAggregate) rel; + RelTraitSet newTrait = deriveTraits(aggregate, aggregate.getTraitSet()); + return new DruidAggregate( + aggregate.getCluster(), + newTrait, + convert(aggregate.getInput(), aggregate.getInput().getTraitSet().replace(DruidLogicalConvention.instance())), + aggregate.getGroupSet(), + aggregate.getGroupSets(), + aggregate.getAggCallList(), + plannerContext + ); + } + + private RelTraitSet deriveTraits(Aggregate aggregate, RelTraitSet traits) + { + final RelCollation collation = traits.getTrait(RelCollationTraitDef.INSTANCE); + if ((collation == null || collation.getFieldCollations().isEmpty()) && aggregate.getGroupSets().size() == 1) { + // Druid sorts by grouping keys when grouping. Add the collation. + // Note: [aggregate.getGroupSets().size() == 1] above means that collation isn't added for GROUPING SETS. + final List sortFields = new ArrayList<>(); + for (int i = 0; i < aggregate.getGroupCount(); i++) { + sortFields.add(new RelFieldCollation(i)); + } + return traits.replace(DruidLogicalConvention.instance()).replace(RelCollations.of(sortFields)); + } + return traits.replace(DruidLogicalConvention.instance()); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidFilterRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidFilterRule.java new file mode 100644 index 00000000000..d67cd17927f --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidFilterRule.java @@ -0,0 +1,62 @@ +/* + * 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.druid.sql.calcite.rule.logical; + +import org.apache.calcite.plan.RelTrait; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.druid.sql.calcite.rel.logical.DruidFilter; +import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; + +/** + * {@link ConverterRule} to convert {@link org.apache.calcite.rel.core.Filter} to {@link DruidFilter} + */ +public class DruidFilterRule extends ConverterRule +{ + + public DruidFilterRule( + Class clazz, + RelTrait in, + RelTrait out, + String descriptionPrefix + ) + { + super(clazz, in, out, descriptionPrefix); + } + + @Override + public RelNode convert(RelNode rel) + { + LogicalFilter filter = (LogicalFilter) rel; + RelTraitSet newTrait = filter.getTraitSet().replace(DruidLogicalConvention.instance()); + return new DruidFilter( + filter.getCluster(), + newTrait, + convert( + filter.getInput(), + filter.getInput().getTraitSet() + .replace(DruidLogicalConvention.instance()) + ), + filter.getCondition() + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidLogicalRules.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidLogicalRules.java new file mode 100644 index 00000000000..d99cdce3d60 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidLogicalRules.java @@ -0,0 +1,90 @@ +/* + * 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.druid.sql.calcite.rule.logical; + +import com.google.common.collect.ImmutableList; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.calcite.rel.logical.LogicalSort; +import org.apache.calcite.rel.logical.LogicalTableScan; +import org.apache.calcite.rel.logical.LogicalValues; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; + +import java.util.ArrayList; +import java.util.List; + + +public class DruidLogicalRules +{ + private final PlannerContext plannerContext; + + public DruidLogicalRules(PlannerContext plannerContext) + { + this.plannerContext = plannerContext; + } + + public List rules() + { + return new ArrayList<>( + ImmutableList.of( + new DruidTableScanRule( + RelOptRule.operand(LogicalTableScan.class, null, RelOptRule.any()), + StringUtils.format("%s", DruidTableScanRule.class.getSimpleName()) + ), + new DruidAggregateRule( + LogicalAggregate.class, + Convention.NONE, + DruidLogicalConvention.instance(), + DruidAggregateRule.class.getSimpleName(), + plannerContext + ), + new DruidSortRule( + LogicalSort.class, + Convention.NONE, + DruidLogicalConvention.instance(), + DruidSortRule.class.getSimpleName() + ), + new DruidProjectRule( + LogicalProject.class, + Convention.NONE, + DruidLogicalConvention.instance(), + DruidProjectRule.class.getSimpleName() + ), + new DruidFilterRule( + LogicalFilter.class, + Convention.NONE, + DruidLogicalConvention.instance(), + DruidFilterRule.class.getSimpleName() + ), + new DruidValuesRule( + LogicalValues.class, + Convention.NONE, + DruidLogicalConvention.instance(), + DruidValuesRule.class.getSimpleName() + ) + ) + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidProjectRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidProjectRule.java new file mode 100644 index 00000000000..00863bee2a9 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidProjectRule.java @@ -0,0 +1,59 @@ +/* + * 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.druid.sql.calcite.rule.logical; + +import org.apache.calcite.plan.RelTrait; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; +import org.apache.druid.sql.calcite.rel.logical.DruidProject; + +/** + * {@link ConverterRule} to convert {@link org.apache.calcite.rel.core.Project} to {@link DruidProject} + */ +public class DruidProjectRule extends ConverterRule +{ + + public DruidProjectRule( + Class clazz, + RelTrait in, + RelTrait out, + String descriptionPrefix + ) + { + super(clazz, in, out, descriptionPrefix); + } + + @Override + public RelNode convert(RelNode rel) + { + LogicalProject project = (LogicalProject) rel; + return DruidProject.create( + convert( + project.getInput(), + project.getInput().getTraitSet() + .replace(DruidLogicalConvention.instance()) + ), + project.getProjects(), + project.getRowType() + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidSortRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidSortRule.java new file mode 100644 index 00000000000..271dd83d74b --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidSortRule.java @@ -0,0 +1,60 @@ +/* + * 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.druid.sql.calcite.rule.logical; + +import org.apache.calcite.plan.RelTrait; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.logical.LogicalSort; +import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; +import org.apache.druid.sql.calcite.rel.logical.DruidSort; + +/** + * {@link ConverterRule} to convert {@link Sort} to {@link DruidSort} + */ +public class DruidSortRule extends ConverterRule +{ + + public DruidSortRule( + Class clazz, + RelTrait in, + RelTrait out, + String descriptionPrefix + ) + { + super(clazz, in, out, descriptionPrefix); + } + + @Override + public RelNode convert(RelNode rel) + { + LogicalSort sort = (LogicalSort) rel; + return DruidSort.create( + convert( + sort.getInput(), + sort.getInput().getTraitSet().replace(DruidLogicalConvention.instance()) + ), + sort.getCollation(), + sort.offset, + sort.fetch + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidTableScanRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidTableScanRule.java new file mode 100644 index 00000000000..517e93f2dc3 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidTableScanRule.java @@ -0,0 +1,54 @@ +/* + * 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.druid.sql.calcite.rule.logical; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptRuleOperand; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.logical.LogicalTableScan; +import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; +import org.apache.druid.sql.calcite.rel.logical.DruidTableScan; + +/** + * {@link ConverterRule} to convert {@link org.apache.calcite.rel.core.TableScan} to {@link DruidTableScan} + */ +public class DruidTableScanRule extends RelOptRule +{ + public DruidTableScanRule(RelOptRuleOperand operand, String description) + { + super(operand, description); + } + + @Override + public void onMatch(RelOptRuleCall call) + { + LogicalTableScan tableScan = call.rel(0); + RelTraitSet newTrait = tableScan.getTraitSet().replace(DruidLogicalConvention.instance()); + DruidTableScan druidTableScan = new DruidTableScan( + tableScan.getCluster(), + newTrait, + tableScan.getTable(), + null + ); + call.transformTo(druidTableScan); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidValuesRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidValuesRule.java new file mode 100644 index 00000000000..5fca4a22967 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidValuesRule.java @@ -0,0 +1,58 @@ +/* + * 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.druid.sql.calcite.rule.logical; + +import org.apache.calcite.plan.RelTrait; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.logical.LogicalValues; +import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; +import org.apache.druid.sql.calcite.rel.logical.DruidValues; + +/** + * {@link ConverterRule} to convert {@link org.apache.calcite.rel.core.Values} to {@link DruidValues} + */ +public class DruidValuesRule extends ConverterRule +{ + + public DruidValuesRule( + Class clazz, + RelTrait in, + RelTrait out, + String descriptionPrefix + ) + { + super(clazz, in, out, descriptionPrefix); + } + + @Override + public RelNode convert(RelNode rel) + { + LogicalValues values = (LogicalValues) rel; + RelTraitSet newTrait = values.getTraitSet().replace(DruidLogicalConvention.instance()); + return new DruidValues( + values.getCluster(), + newTrait, + values.getRowType(), + values.getTuples() + ); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 428e1d82004..b93228e076f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -858,6 +858,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase public class CalciteTestConfig implements QueryTestBuilder.QueryTestConfig { private boolean isRunningMSQ = false; + private Map baseQueryContext; public CalciteTestConfig() { @@ -868,6 +869,11 @@ public class BaseCalciteQueryTest extends CalciteTestBase this.isRunningMSQ = isRunningMSQ; } + public CalciteTestConfig(Map baseQueryContext) + { + this.baseQueryContext = baseQueryContext; + } + @Override public QueryLogHook queryLogHook() { @@ -909,6 +915,12 @@ public class BaseCalciteQueryTest extends CalciteTestBase { return isRunningMSQ; } + + @Override + public Map baseQueryContext() + { + return baseQueryContext; + } } public void assertResultsEquals(String sql, List expectedResults, List results) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java new file mode 100644 index 00000000000..dfd1acad0cf --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java @@ -0,0 +1,425 @@ +/* + * 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.druid.sql.calcite; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.sql.calcite.planner.PlannerConfig; +import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.junit.Ignore; +import org.junit.Test; + +public class DecoupledPlanningCalciteQueryTest extends CalciteQueryTest +{ + private static final ImmutableMap CONTEXT_OVERRIDES = ImmutableMap.of( + PlannerConfig.CTX_NATIVE_QUERY_SQL_PLANNING_MODE, PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED, + QueryContexts.ENABLE_DEBUG, true + ); + + @Override + protected QueryTestBuilder testBuilder() + { + return new QueryTestBuilder( + new CalciteTestConfig(CONTEXT_OVERRIDES) + { + @Override + public SqlTestFramework.PlannerFixture plannerFixture(PlannerConfig plannerConfig, AuthConfig authConfig) + { + plannerConfig = plannerConfig.withOverrides(CONTEXT_OVERRIDES); + return queryFramework().plannerFixture(DecoupledPlanningCalciteQueryTest.this, plannerConfig, authConfig); + } + }) + .cannotVectorize(cannotVectorize) + .skipVectorize(skipVectorize); + } + + + @Override + @Ignore + public void testGroupByWithSelectAndOrderByProjections() + { + + } + + @Override + @Ignore + public void testTopNWithSelectAndOrderByProjections() + { + + } + + @Override + @Ignore + public void testUnionAllQueries() + { + + } + + @Override + @Ignore + public void testUnionAllQueriesWithLimit() + { + + } + + @Override + @Ignore + public void testUnionAllDifferentTablesWithMapping() + { + + } + + @Override + @Ignore + public void testJoinUnionAllDifferentTablesWithMapping() + { + + } + + @Override + @Ignore + public void testUnionAllTablesColumnTypeMismatchFloatLong() + { + + } + + @Override + @Ignore + public void testUnionAllTablesColumnTypeMismatchStringLong() + { + + } + + @Override + @Ignore + public void testUnionAllTablesWhenMappingIsRequired() + { + + } + + @Override + @Ignore + public void testUnionIsUnplannable() + { + + } + + @Override + @Ignore + public void testUnionAllTablesWhenCastAndMappingIsRequired() + { + + } + + @Override + @Ignore + public void testUnionAllSameTableTwice() + { + + } + + @Override + @Ignore + public void testUnionAllSameTableTwiceWithSameMapping() + { + + } + + @Override + @Ignore + public void testUnionAllSameTableTwiceWithDifferentMapping() + { + + } + + @Override + @Ignore + public void testUnionAllSameTableThreeTimes() + { + + } + + @Override + @Ignore + public void testUnionAllSameTableThreeTimesWithSameMapping() + { + + } + + @Override + @Ignore + public void testSelfJoin() + { + + } + + @Override + @Ignore + public void testTwoExactCountDistincts() + { + + } + + @Override + @Ignore + public void testViewAndJoin() + { + + } + + @Override + @Ignore + public void testGroupByWithSortOnPostAggregationDefault() + { + + } + + @Override + @Ignore + public void testGroupByWithSortOnPostAggregationNoTopNConfig() + { + + } + + @Override + @Ignore + public void testGroupByWithSortOnPostAggregationNoTopNContext() + { + + } + + @Override + @Ignore + public void testUnplannableQueries() + { + + } + + @Override + @Ignore + public void testUnplannableTwoExactCountDistincts() + { + + } + + @Override + @Ignore + public void testUnplannableExactCountDistinctOnSketch() + { + + } + + @Override + @Ignore + public void testExactCountDistinctUsingSubqueryOnUnionAllTables() + { + + } + + @Override + @Ignore + public void testUseTimeFloorInsteadOfGranularityOnJoinResult() + { + + } + + @Override + @Ignore + public void testMinMaxAvgDailyCountWithLimit() + { + + } + + @Override + @Ignore + public void testExactCountDistinctOfSemiJoinResult() + { + + } + + @Override + @Ignore + public void testMaxSubqueryRows() + { + + } + + @Override + @Ignore + public void testExactCountDistinctUsingSubqueryWithWherePushDown() + { + + } + + @Override + @Ignore + public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() + { + + } + + @Override + @Ignore + public void testUsingSubqueryAsFilterOnTwoColumns() + { + + } + + @Override + @Ignore + public void testUsingSubqueryAsFilterWithInnerSort() + { + + } + + @Override + @Ignore + public void testUsingSubqueryWithLimit() + { + + } + + @Override + @Ignore + public void testPostAggWithTimeseries() + { + + } + + @Override + @Ignore + public void testPostAggWithTopN() + { + + } + + @Override + @Ignore + public void testRequireTimeConditionPositive() + { + + } + + @Override + public void testRequireTimeConditionSemiJoinNegative() + { + + } + + @Override + @Ignore + public void testEmptyGroupWithOffsetDoesntInfiniteLoop() + { + + } + + @Override + @Ignore + public void testJoinWithTimeDimension() + { + + } + + @Override + @Ignore + public void testSubqueryTypeMismatchWithLiterals() + { + + } + + @Override + @Ignore + public void testTimeseriesQueryWithEmptyInlineDatasourceAndGranularity() + { + + } + + @Override + @Ignore + public void testGroupBySortPushDown() + { + + } + + @Override + @Ignore + public void testGroupingWithNullInFilter() + { + + } + + @Override + @Ignore + @Test + public void testStringAggExpressionNonConstantSeparator() + { + + } + + @Override + @Ignore + public void testOrderByAlongWithInternalScanQuery() + { + + } + + @Override + @Ignore + public void testSortProjectAfterNestedGroupBy() + { + + } + + @Override + @Ignore + public void testOrderByAlongWithInternalScanQueryNoDistinct() + { + + } + + @Override + @Ignore + public void testNestedGroupBy() + { + + } + + @Override + @Ignore + public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename() + { + + } + + @Override + @Ignore + public void testFilterOnCurrentTimestampWithIntervalArithmetic() + { + + } + + @Override + @Ignore + public void testFilterOnCurrentTimestampOnView() + { + + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java index 187beab91dd..d5e20043adc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java @@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; @@ -79,11 +80,13 @@ public class QueryTestBuilder ResultsVerifier defaultResultsVerifier(List expectedResults, RowSignature expectedResultSignature); boolean isRunningMSQ(); + + Map baseQueryContext(); } protected final QueryTestConfig config; protected PlannerConfig plannerConfig = BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT; - protected Map queryContext = BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT; + protected Map queryContext; protected List parameters = CalciteTestBase.DEFAULT_PARAMETERS; protected String sql; protected AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT; @@ -108,6 +111,12 @@ public class QueryTestBuilder public QueryTestBuilder(final QueryTestConfig config) { this.config = config; + // Done to maintain backwards compat. So, + // 1. If no base context is provided in config, the queryContext is set to the default one + // 2. If some base context is provided in config, we set that context as the queryContext + // 3. If someone overrides the context, we merge the context with the empty/non-empty base context provided in the config + this.queryContext = + config.baseQueryContext() == null ? BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT : config.baseQueryContext(); } public QueryTestBuilder plannerConfig(PlannerConfig plannerConfig) @@ -118,7 +127,7 @@ public class QueryTestBuilder public QueryTestBuilder queryContext(Map queryContext) { - this.queryContext = queryContext; + this.queryContext = QueryContexts.override(config.baseQueryContext(), queryContext); return this; } From f5cc823d0f8acdf660860fcb75db34e38c039d8f Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 20 Jun 2023 15:25:57 +0530 Subject: [PATCH 09/74] Handle nulls in DruidCoordinator.getReplicationFactor (#14447) --- .../apache/druid/server/coordinator/DruidCoordinator.java | 6 +++--- .../druid/server/coordinator/DruidCoordinatorTest.java | 6 ++++++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 32c56bd0329..0c5e79bdcbe 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -302,11 +302,11 @@ public class DruidCoordinator @Nullable public Integer getReplicationFactor(SegmentId segmentId) { - if (segmentReplicationStatus != null) { - return segmentReplicationStatus.getReplicaCountsInCluster(segmentId).required(); - } else { + if (segmentReplicationStatus == null) { return null; } + SegmentReplicaCount replicaCountsInCluster = segmentReplicationStatus.getReplicaCountsInCluster(segmentId); + return replicaCountsInCluster == null ? null : replicaCountsInCluster.required(); } @Nullable diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index dd442405cd6..6c0ab813b37 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -249,6 +249,9 @@ public class DruidCoordinatorTest extends CuratorTestBase EasyMock.replay(serverInventoryView, loadQueueTaskMaster); coordinator.start(); + + Assert.assertNull(coordinator.getReplicationFactor(dataSegment.getId())); + // Wait for this coordinator to become leader leaderAnnouncerLatch.await(); @@ -306,6 +309,7 @@ public class DruidCoordinatorTest extends CuratorTestBase // the segments are replicated as many times as they can be given state of cluster, therefore should not be // under-replicated. Assert.assertEquals(0L, underRepliicationCountsPerDataSourceUsingClusterView.getLong(dataSource)); + Assert.assertEquals(Integer.valueOf(2), coordinator.getReplicationFactor(dataSegment.getId())); coordinator.stop(); leaderUnannouncerLatch.await(); @@ -398,6 +402,8 @@ public class DruidCoordinatorTest extends CuratorTestBase Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(hotTierName).getLong(dataSource)); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(coldTierName).getLong(dataSource)); + dataSegments.values().forEach(dataSegment -> Assert.assertEquals(Integer.valueOf(1), coordinator.getReplicationFactor(dataSegment.getId()))); + coordinator.stop(); leaderUnannouncerLatch.await(); From 1ea9158a50ecb3c6e928c8bbc85bdade9ab80ed6 Mon Sep 17 00:00:00 2001 From: Hardik Bajaj <58038410+hardikbajaj@users.noreply.github.com> Date: Tue, 20 Jun 2023 20:57:58 +0530 Subject: [PATCH 10/74] Added new SysMonitorOshi v0 using Oshi library (#14359) Added a new monitor SysMonitorOshi to replace SysMonitor. The new monitor has a wider support for different machine architectures including ARM instances. Please switch to SysMonitorOshi as SysMonitor is now deprecated and will be removed in future releases. --- LICENSE | 3 + distribution/bin/check-licenses.py | 2 +- licenses.yaml | 15 +- licenses/bin/oshi.MIT | 21 + pom.xml | 9 +- processing/pom.xml | 6 + .../java/util/metrics/NoopOshiSysMonitor.java | 36 ++ .../java/util/metrics/OshiSysMonitor.java | 466 +++++++++++++ .../druid/java/util/metrics/SysMonitor.java | 8 + .../util/metrics/NoopOshiSysMonitorTest.java | 38 ++ .../java/util/metrics/OshiSysMonitorTest.java | 611 ++++++++++++++++++ .../druid/server/metrics/MetricsModule.java | 17 + .../server/metrics/MetricsModuleTest.java | 26 + 13 files changed, 1254 insertions(+), 4 deletions(-) create mode 100644 licenses/bin/oshi.MIT create mode 100644 processing/src/main/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitor.java create mode 100644 processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java create mode 100644 processing/src/test/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitorTest.java create mode 100644 processing/src/test/java/org/apache/druid/java/util/metrics/OshiSysMonitorTest.java diff --git a/LICENSE b/LICENSE index 68531acb2de..4b63a77502e 100644 --- a/LICENSE +++ b/LICENSE @@ -279,6 +279,9 @@ SOURCE/JAVA-CORE This product contains lpad and rpad methods adapted from Apache Flink. * processing/src/main/java/org/apache/druid/java/util/common/StringUtils.java + This product contains SystemInfo methods adapted from oshi + * processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java + MIT License ================================ diff --git a/distribution/bin/check-licenses.py b/distribution/bin/check-licenses.py index ff77eeace0b..5b059fd23af 100755 --- a/distribution/bin/check-licenses.py +++ b/distribution/bin/check-licenses.py @@ -292,6 +292,7 @@ def build_compatible_license_names(): compatible_licenses['MIT License'] = 'MIT License' compatible_licenses['The MIT License (MIT)'] = 'MIT License' compatible_licenses['Bouncy Castle Licence'] = 'MIT License' + compatible_licenses['SPDX-License-Identifier: MIT'] = 'MIT License' compatible_licenses['The Go license'] = 'The Go license' @@ -435,7 +436,6 @@ if __name__ == "__main__": license_yaml = args.license_yaml dependency_reports_root = args.dependency_reports_root - check_licenses(license_yaml, dependency_reports_root) except KeyboardInterrupt: diff --git a/licenses.yaml b/licenses.yaml index a16bdc80e9f..843fc685747 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -1591,7 +1591,7 @@ name: Java Native Access (JNA) license_category: binary module: java-core license_name: Apache License version 2.0 -version: 4.5.1 +version: 5.13.0 libraries: - net.java.dev.jna: jna @@ -2341,6 +2341,17 @@ notices: --- +name: OSHI +license_category: binary +module: java-core +license_name: MIT License +version: 6.4.2 +libraries: + - com.github.oshi: oshi-core +license_file_path: licenses/bin/oshi.MIT + +--- + name: JBoss Logging 3 license_category: binary module: java-core @@ -4914,7 +4925,7 @@ libraries: name: net.java.dev.jna jna-platform license_category: binary -version: 5.2.0 +version: 5.13.0 module: druid-ranger-security license_name: Apache License version 2.0 libraries: diff --git a/licenses/bin/oshi.MIT b/licenses/bin/oshi.MIT new file mode 100644 index 00000000000..5fa8e283344 --- /dev/null +++ b/licenses/bin/oshi.MIT @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2010-2023 The OSHI Project Contributors: https://github.com/oshi/oshi/graphs/contributors + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. \ No newline at end of file diff --git a/pom.xml b/pom.xml index 7e50bac3a58..c495ea3255a 100644 --- a/pom.xml +++ b/pom.xml @@ -109,6 +109,8 @@ 3.21.7 1.3.1 1.7.36 + 5.13.0 + 5.13.0 3.3.5 4.3.1 1.12.317 @@ -882,7 +884,12 @@ net.java.dev.jna jna - 4.5.1 + ${jna.version} + + + net.java.dev.jna + jna-platform + ${jna-platform.version} org.apache.commons diff --git a/processing/pom.xml b/processing/pom.xml index 290523b2705..c41622d9c1e 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -37,6 +37,7 @@ 1.6.5 ${sigar.base.version}.132 5.3.4 + 6.4.2 @@ -335,6 +336,11 @@ dependency are copied as resources. See maven-dependency-plugin configuration and below. --> provided + + com.github.oshi + oshi-core + ${oshi.version} + diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitor.java b/processing/src/main/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitor.java new file mode 100644 index 00000000000..d44390ac8c1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitor.java @@ -0,0 +1,36 @@ +/* + * 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.druid.java.util.metrics; + +import org.apache.druid.java.util.emitter.service.ServiceEmitter; + +public class NoopOshiSysMonitor extends OshiSysMonitor +{ + public NoopOshiSysMonitor() + { + super(); + } + + @Override + public boolean doMonitor(ServiceEmitter emitter) + { + return false; + } +} diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java b/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java new file mode 100644 index 00000000000..40d97b57c7e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java @@ -0,0 +1,466 @@ +/* + * 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.druid.java.util.metrics; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import oshi.SystemInfo; +import oshi.hardware.CentralProcessor; +import oshi.hardware.CentralProcessor.TickType; +import oshi.hardware.GlobalMemory; +import oshi.hardware.HWDiskStore; +import oshi.hardware.HardwareAbstractionLayer; +import oshi.hardware.NetworkIF; +import oshi.hardware.VirtualMemory; +import oshi.software.os.FileSystem; +import oshi.software.os.InternetProtocolStats; +import oshi.software.os.OSFileStore; +import oshi.software.os.OperatingSystem; + +import java.util.List; +import java.util.Map; + +/** + * SysMonitor implemented using {@link oshi} + *

+ * Following stats are emitted: + *

    + *
  • {@link MemStats} for Memory related metrics
  • + *
  • {@link SwapStats} for swap storage related metrics
  • + *
  • {@link FsStats} for File System related Metrics
  • + *
  • {@link DiskStats} for Disk level metrics
  • + *
  • {@link NetStats} for Network Interface and related metrics
  • + *
  • {@link CpuStats} for CPU usage and stats metrics
  • + *
  • {@link SysStats} for overall system metrics(uptime, avg load)
  • + *
  • {@link TcpStats} for TCP related metrics
  • + *
+ */ +public class OshiSysMonitor extends FeedDefiningMonitor +{ + + private final SystemInfo si; + private final HardwareAbstractionLayer hal; + private final OperatingSystem os; + private static final List NET_ADDRESS_BLACKLIST = ImmutableList.of("0.0.0.0", "127.0.0.1"); + private final MemStats memStats; + private final SwapStats swapStats; + private final FsStats fsStats; + private final DiskStats diskStats; + private final NetStats netStats; + private final CpuStats cpuStats; + private final SysStats sysStats; + private final TcpStats tcpStats; + + private final Map dimensions; + + public OshiSysMonitor() + { + this(ImmutableMap.of()); + } + + public OshiSysMonitor(Map dimensions) + { + this(dimensions, DEFAULT_METRICS_FEED); + } + + public OshiSysMonitor(Map dimensions, String feed) + { + super(feed); + Preconditions.checkNotNull(dimensions); + this.dimensions = ImmutableMap.copyOf(dimensions); + + this.si = new SystemInfo(); + this.hal = si.getHardware(); + this.os = si.getOperatingSystem(); + + this.memStats = new MemStats(); + this.swapStats = new SwapStats(); + this.fsStats = new FsStats(); + this.diskStats = new DiskStats(); + this.netStats = new NetStats(); + this.cpuStats = new CpuStats(); + this.sysStats = new SysStats(); + this.tcpStats = new TcpStats(); + + } + + // Create an object with mocked systemInfo for testing purposes + public OshiSysMonitor(SystemInfo systemInfo) + { + super("metrics"); + this.dimensions = ImmutableMap.of(); + + this.si = systemInfo; + this.hal = si.getHardware(); + this.os = si.getOperatingSystem(); + + this.memStats = new MemStats(); + this.swapStats = new SwapStats(); + this.fsStats = new FsStats(); + this.diskStats = new DiskStats(); + this.netStats = new NetStats(); + this.cpuStats = new CpuStats(); + this.sysStats = new SysStats(); + this.tcpStats = new TcpStats(); + } + + @Override + public boolean doMonitor(ServiceEmitter emitter) + { + monitorMemStats(emitter); + monitorSwapStats(emitter); + monitorFsStats(emitter); + monitorDiskStats(emitter); + monitorNetStats(emitter); + monitorCpuStats(emitter); + monitorSysStats(emitter); + monitorTcpStats(emitter); + return true; + } + + // Emit stats for a particular stat(mem, swap, filestore, etc) from statsList for testing + public void monitorMemStats(ServiceEmitter emitter) + { + memStats.emit(emitter); + } + + public void monitorSwapStats(ServiceEmitter emitter) + { + swapStats.emit(emitter); + } + + public void monitorFsStats(ServiceEmitter emitter) + { + fsStats.emit(emitter); + } + + public void monitorDiskStats(ServiceEmitter emitter) + { + diskStats.emit(emitter); + } + + public void monitorNetStats(ServiceEmitter emitter) + { + netStats.emit(emitter); + } + + public void monitorCpuStats(ServiceEmitter emitter) + { + cpuStats.emit(emitter); + } + + public void monitorSysStats(ServiceEmitter emitter) + { + sysStats.emit(emitter); + } + + public void monitorTcpStats(ServiceEmitter emitter) + { + tcpStats.emit(emitter); + } + + /** + * Implementation of Memstats + *

+ * Define a method {@link #emit(ServiceEmitter)} to emit metrices in emiters + */ + + private class MemStats + { + public void emit(ServiceEmitter emitter) + { + GlobalMemory mem = hal.getMemory(); + if (mem != null) { + final Map stats = ImmutableMap.of( + "sys/mem/max", + mem.getTotal(), + "sys/mem/used", + mem.getTotal() - mem.getAvailable(), + // This is total actual memory used, not including cache and buffer memory + "sys/mem/free", + mem.getAvailable() + ); + final ServiceMetricEvent.Builder builder = builder(); + MonitorUtils.addDimensionsToBuilder(builder, dimensions); + for (Map.Entry entry : stats.entrySet()) { + emitter.emit(builder.build(entry.getKey(), entry.getValue())); + } + } + } + } + + private class SwapStats + { + private long prevPageIn = 0; + private long prevPageOut = 0; + + public void emit(ServiceEmitter emitter) + { + VirtualMemory swap = hal.getMemory().getVirtualMemory(); + + if (swap != null) { + long currPageIn = swap.getSwapPagesIn(); + long currPageOut = swap.getSwapPagesOut(); + final Map stats = ImmutableMap.of( + "sys/swap/pageIn", currPageIn - prevPageIn, + "sys/swap/pageOut", currPageOut - prevPageOut, + "sys/swap/max", swap.getSwapTotal(), + "sys/swap/free", swap.getSwapTotal() - swap.getSwapUsed() + ); + + final ServiceMetricEvent.Builder builder = builder(); + MonitorUtils.addDimensionsToBuilder(builder, dimensions); + for (Map.Entry entry : stats.entrySet()) { + emitter.emit(builder.build(entry.getKey(), entry.getValue())); + } + + this.prevPageIn = currPageIn; + this.prevPageOut = currPageOut; + } + } + } + + private class FsStats + { + public void emit(ServiceEmitter emitter) + { + FileSystem fileSystem = os.getFileSystem(); + for (OSFileStore fs : fileSystem.getFileStores(true)) { // get only local file store : true + + final Map stats = ImmutableMap.builder() + .put("sys/fs/max", fs.getTotalSpace()) + .put("sys/fs/used", fs.getTotalSpace() - fs.getUsableSpace()) + .put("sys/fs/files/count", fs.getTotalInodes()) + .put("sys/fs/files/free", fs.getFreeInodes()) + .build(); + final ServiceMetricEvent.Builder builder = builder() + .setDimension("fsDevName", fs.getVolume()) + .setDimension("fsDirName", fs.getMount()); + MonitorUtils.addDimensionsToBuilder(builder, dimensions); + for (Map.Entry entry : stats.entrySet()) { + emitter.emit(builder.build(entry.getKey(), entry.getValue())); + } + } + } + } + + private class DiskStats + { + // Difference b/w metrics of two consecutive values. It tells Δmetric (increase/decrease in metrics value) + private final KeyedDiff diff = new KeyedDiff(); + + public void emit(ServiceEmitter emitter) + { + List disks = hal.getDiskStores(); + // disk partitions can be mapped to file system but no inbuilt method is there to find relation b/w disks and file system + // Will have to add logic for that + for (HWDiskStore disk : disks) { + + final Map stats = diff.to( + disk.getName(), + ImmutableMap.builder() + .put("sys/disk/read/size", disk.getReadBytes()) + .put("sys/disk/read/count", disk.getReads()) + .put("sys/disk/write/size", disk.getWriteBytes()) + .put("sys/disk/write/count", disk.getWrites()) + .put("sys/disk/queue", disk.getCurrentQueueLength()) + .put("sys/disk/transferTime", disk.getTransferTime()) + .build() + ); + if (stats != null) { + final ServiceMetricEvent.Builder builder = builder() + .setDimension("diskName", disk.getName()); + MonitorUtils.addDimensionsToBuilder(builder, dimensions); + for (Map.Entry entry : stats.entrySet()) { + emitter.emit(builder.build(entry.getKey(), entry.getValue())); + } + } + } + } + } + + private class NetStats + { + private final KeyedDiff diff = new KeyedDiff(); + + public void emit(ServiceEmitter emitter) + { + List networkIFS = hal.getNetworkIFs(); + for (NetworkIF net : networkIFS) { + final String name = net.getName(); + for (String addr : net.getIPv4addr()) { + if (!NET_ADDRESS_BLACKLIST.contains(addr)) { + // Only emit metrics for non black-listed ip addresses + String mapKey = name + + "_" + + addr; // Network_Name_IPV4 address as key, ex: wifi_192.1.0.1 to uniquely identify the dimension + final Map stats = diff.to( + mapKey, + ImmutableMap.builder() + .put("sys/net/read/size", net.getBytesRecv()) + .put("sys/net/read/packets", net.getPacketsRecv()) + .put("sys/net/read/errors", net.getInErrors()) + .put("sys/net/read/dropped", net.getInDrops()) + .put("sys/net/write/size", net.getBytesSent()) + .put("sys/net/write/packets", net.getPacketsSent()) + .put("sys/net/write/errors", net.getOutErrors()) + .put("sys/net/write/collisions", net.getCollisions()) + .build() + ); + if (stats != null) { + final ServiceMetricEvent.Builder builder = builder() + .setDimension("netName", net.getName()) + .setDimension("netAddress", addr) + .setDimension("netHwaddr", net.getMacaddr()); + MonitorUtils.addDimensionsToBuilder(builder, dimensions); + for (Map.Entry entry : stats.entrySet()) { + emitter.emit(builder.build(entry.getKey(), entry.getValue())); + } + } + } + } + } + } + } + + private class CpuStats + { + private final KeyedDiff diff = new KeyedDiff(); + + + public void emit(ServiceEmitter emitter) + { + CentralProcessor processor = hal.getProcessor(); + long[][] procTicks = processor.getProcessorCpuLoadTicks(); + for (int i = 0; i < procTicks.length; ++i) { + final String name = Integer.toString(i); + long[] ticks = procTicks[i]; + long user = ticks[TickType.USER.getIndex()]; + long nice = ticks[TickType.NICE.getIndex()]; + long sys = ticks[TickType.SYSTEM.getIndex()]; + long idle = ticks[TickType.IDLE.getIndex()]; + long iowait = ticks[TickType.IOWAIT.getIndex()]; + long irq = ticks[TickType.IRQ.getIndex()]; + long softirq = ticks[TickType.SOFTIRQ.getIndex()]; + long steal = ticks[TickType.STEAL.getIndex()]; + long totalCpu = user + nice + sys + idle + iowait + irq + softirq + steal; + final Map stats = diff.to( + name, + ImmutableMap.builder() + .put("user", user) // user = Δuser / Δtotal + .put("sys", sys) // sys = Δsys / Δtotal + .put("nice", nice) // nice = Δnice / Δtotal + .put("wait", iowait) // wait = Δwait / Δtotal + .put("irq", irq) // irq = Δirq / Δtotal + .put("softIrq", softirq) // softIrq = ΔsoftIrq / Δtotal + .put("stolen", steal) // stolen = Δstolen / Δtotal + .put("idle", idle) // idle = Δidle / Δtotal + .put("_total", totalCpu) // (not reported) + .build() + ); + if (stats != null) { + final long total = stats.remove("_total"); + for (Map.Entry entry : stats.entrySet()) { + final ServiceMetricEvent.Builder builder = builder() + .setDimension("cpuName", name) + .setDimension("cpuTime", entry.getKey()); + MonitorUtils.addDimensionsToBuilder(builder, dimensions); + if (total != 0) { + // prevent divide by 0 exception and don't emit such events + emitter.emit(builder.build("sys/cpu", entry.getValue() * 100 / total)); // [0,100] + } + + } + } + } + } + } + + private class SysStats + { + + public void emit(ServiceEmitter emitter) + { + final ServiceMetricEvent.Builder builder = builder(); + MonitorUtils.addDimensionsToBuilder(builder, dimensions); + + long uptime = os.getSystemUptime(); + + final Map stats = ImmutableMap.of( + "sys/uptime", uptime + ); + for (Map.Entry entry : stats.entrySet()) { + emitter.emit(builder.build(entry.getKey(), entry.getValue())); + } + CentralProcessor processor = hal.getProcessor(); + double[] la = processor.getSystemLoadAverage(3); + + if (la != null) { + final Map statsCpuLoadAverage = ImmutableMap.of( + "sys/la/1", la[0], + "sys/la/5", la[1], + "sys/la/15", la[2] + ); + for (Map.Entry entry : statsCpuLoadAverage.entrySet()) { + emitter.emit(builder.build(entry.getKey(), entry.getValue())); + } + } + } + } + + private class TcpStats + { + private final KeyedDiff diff = new KeyedDiff(); + + public void emit(ServiceEmitter emitter) + { + final ServiceMetricEvent.Builder builder = builder(); + MonitorUtils.addDimensionsToBuilder(builder, dimensions); + + InternetProtocolStats ipstats = os.getInternetProtocolStats(); + InternetProtocolStats.TcpStats tcpv4 = ipstats.getTCPv4Stats(); + + if (tcpv4 != null) { + final Map stats = diff.to( + "tcpv4", ImmutableMap.builder() + .put("sys/tcpv4/activeOpens", tcpv4.getConnectionsActive()) + .put("sys/tcpv4/passiveOpens", tcpv4.getConnectionsPassive()) + .put("sys/tcpv4/attemptFails", tcpv4.getConnectionFailures()) + .put("sys/tcpv4/estabResets", tcpv4.getConnectionsReset()) + .put("sys/tcpv4/in/segs", tcpv4.getSegmentsReceived()) + .put("sys/tcpv4/in/errs", tcpv4.getInErrors()) + .put("sys/tcpv4/out/segs", tcpv4.getSegmentsSent()) + .put("sys/tcpv4/out/rsts", tcpv4.getOutResets()) + .put("sys/tcpv4/retrans/segs", tcpv4.getSegmentsRetransmitted()) + .build() + ); + if (stats != null) { + for (Map.Entry entry : stats.entrySet()) { + emitter.emit(builder.build(entry.getKey(), entry.getValue())); + } + } + } + } + } + +} diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/SysMonitor.java b/processing/src/main/java/org/apache/druid/java/util/metrics/SysMonitor.java index d161bc1ad05..c8ce4cfb90b 100644 --- a/processing/src/main/java/org/apache/druid/java/util/metrics/SysMonitor.java +++ b/processing/src/main/java/org/apache/druid/java/util/metrics/SysMonitor.java @@ -46,6 +46,14 @@ import java.util.Arrays; import java.util.List; import java.util.Map; + +/** + * Deprecated, SysMonitor will now be maintained in {@link OshiSysMonitor} + * + * Sys monitor was implemented using @link org.hyperic.sigar which is no longer maintained. + * {@link oshi} based SysMonitor will be maintained and used from now on, and is implemented in org.apache.druid.java.util.metrics.OshiSysMonitor + */ +@Deprecated public class SysMonitor extends FeedDefiningMonitor { private static final Logger log = new Logger(SysMonitor.class); diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitorTest.java b/processing/src/test/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitorTest.java new file mode 100644 index 00000000000..d07f160e218 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitorTest.java @@ -0,0 +1,38 @@ +/* + * 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.druid.java.util.metrics; + +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +public class NoopOshiSysMonitorTest +{ + @Test + public void testDoMonitor() + { + + ServiceEmitter serviceEmitter = Mockito.mock(ServiceEmitter.class); + NoopOshiSysMonitor noopOshiSysMonitor = new NoopOshiSysMonitor(); + + Assert.assertFalse(noopOshiSysMonitor.doMonitor(serviceEmitter)); + } +} diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/OshiSysMonitorTest.java b/processing/src/test/java/org/apache/druid/java/util/metrics/OshiSysMonitorTest.java new file mode 100644 index 00000000000..3b86b1efd76 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/java/util/metrics/OshiSysMonitorTest.java @@ -0,0 +1,611 @@ +/* + * 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.druid.java.util.metrics; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; +import oshi.SystemInfo; +import oshi.hardware.CentralProcessor; +import oshi.hardware.GlobalMemory; +import oshi.hardware.HWDiskStore; +import oshi.hardware.HardwareAbstractionLayer; +import oshi.hardware.NetworkIF; +import oshi.hardware.VirtualMemory; +import oshi.software.os.FileSystem; +import oshi.software.os.InternetProtocolStats; +import oshi.software.os.OSFileStore; +import oshi.software.os.OperatingSystem; +import oshi.util.Util; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class OshiSysMonitorTest +{ + + private SystemInfo si; + private HardwareAbstractionLayer hal; + private OperatingSystem os; + + private enum STATS + { + MEM, SWAP, FS, DISK, NET, CPU, SYS, TCP + } + + @Before + public void setUp() + { + si = Mockito.mock(SystemInfo.class); + hal = Mockito.mock(HardwareAbstractionLayer.class); + os = Mockito.mock(OperatingSystem.class); + Mockito.when(si.getHardware()).thenReturn(hal); + Mockito.when(si.getOperatingSystem()).thenReturn(os); + } + + @Test + public void testDoMonitor() + { + + ServiceEmitter serviceEmitter = Mockito.mock(ServiceEmitter.class); + OshiSysMonitor sysMonitorOshi = new OshiSysMonitor(); + serviceEmitter.start(); + sysMonitorOshi.monitor(serviceEmitter); + + Assert.assertTrue(sysMonitorOshi.doMonitor(serviceEmitter)); + + } + + @Test + public void testDefaultFeedSysMonitorOshi() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + OshiSysMonitor m = new OshiSysMonitor(); + m.start(); + m.monitor(emitter); + // Sleep for 2 sec to get all metrics which are difference of prev and now metrics + Util.sleep(2000); + m.monitor(emitter); + m.stop(); + checkEvents(emitter.getEvents(), "metrics"); + } + + @Test + public void testMemStats() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + GlobalMemory mem = Mockito.mock(GlobalMemory.class); + Mockito.when(mem.getTotal()).thenReturn(64L); + Mockito.when(mem.getAvailable()).thenReturn(16L); + Mockito.when(hal.getMemory()).thenReturn(mem); + + OshiSysMonitor m = new OshiSysMonitor(si); + m.start(); + m.monitorMemStats(emitter); + m.stop(); + Assert.assertEquals(3, emitter.getEvents().size()); + emitter.verifyEmitted("sys/mem/max", 1); + emitter.verifyEmitted("sys/mem/used", 1); + emitter.verifyEmitted("sys/mem/free", 1); + emitter.verifyValue("sys/mem/max", 64L); + emitter.verifyValue("sys/mem/used", 48L); + emitter.verifyValue("sys/mem/free", 16L); + } + + @Test + public void testSwapStats() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + GlobalMemory mem = Mockito.mock(GlobalMemory.class); + VirtualMemory swap = Mockito.mock(VirtualMemory.class); + Mockito.when(swap.getSwapPagesIn()).thenReturn(300L); + Mockito.when(swap.getSwapPagesOut()).thenReturn(200L); + Mockito.when(swap.getSwapTotal()).thenReturn(1000L); + Mockito.when(swap.getSwapUsed()).thenReturn(700L); + Mockito.when(mem.getVirtualMemory()).thenReturn(swap); + Mockito.when(hal.getMemory()).thenReturn(mem); + + OshiSysMonitor m = new OshiSysMonitor(si); + m.start(); + m.monitorSwapStats(emitter); + Assert.assertEquals(4, emitter.getEvents().size()); + emitter.verifyEmitted("sys/swap/pageIn", 1); + emitter.verifyEmitted("sys/swap/pageOut", 1); + emitter.verifyEmitted("sys/swap/max", 1); + emitter.verifyEmitted("sys/swap/free", 1); + emitter.verifyValue("sys/swap/pageIn", 300L); + emitter.verifyValue("sys/swap/pageOut", 200L); + emitter.verifyValue("sys/swap/max", 1000L); + emitter.verifyValue("sys/swap/free", 300L); + // Emit again to assert diff in pageIn stats + Mockito.when(swap.getSwapPagesIn()).thenReturn(400L); + Mockito.when(swap.getSwapPagesOut()).thenReturn(250L); + Mockito.when(swap.getSwapUsed()).thenReturn(500L); + emitter.flush(); + m.monitorSwapStats(emitter); + emitter.verifyValue("sys/swap/pageIn", 100L); + emitter.verifyValue("sys/swap/pageOut", 50L); + emitter.verifyValue("sys/swap/max", 1000L); + emitter.verifyValue("sys/swap/free", 500L); + m.stop(); + } + + @Test + public void testFsStats() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + FileSystem fileSystem = Mockito.mock(FileSystem.class); + OSFileStore fs1 = Mockito.mock(OSFileStore.class); + OSFileStore fs2 = Mockito.mock(OSFileStore.class); + Mockito.when(fs1.getTotalSpace()).thenReturn(300L); + Mockito.when(fs1.getUsableSpace()).thenReturn(200L); + Mockito.when(fs1.getTotalInodes()).thenReturn(1000L); + Mockito.when(fs1.getFreeInodes()).thenReturn(700L); + Mockito.when(fs1.getVolume()).thenReturn("/dev/disk1"); + Mockito.when(fs1.getMount()).thenReturn("/System/Volumes/boot1"); + Mockito.when(fs2.getTotalSpace()).thenReturn(400L); + Mockito.when(fs2.getUsableSpace()).thenReturn(320L); + Mockito.when(fs2.getTotalInodes()).thenReturn(800L); + Mockito.when(fs2.getFreeInodes()).thenReturn(600L); + Mockito.when(fs2.getVolume()).thenReturn("/dev/disk2"); + Mockito.when(fs2.getMount()).thenReturn("/System/Volumes/boot2"); + List osFileStores = ImmutableList.of(fs1, fs2); + Mockito.when(fileSystem.getFileStores(true)).thenReturn(osFileStores); + Mockito.when(os.getFileSystem()).thenReturn(fileSystem); + + OshiSysMonitor m = new OshiSysMonitor(si); + m.start(); + m.monitorFsStats(emitter); + Assert.assertEquals(8, emitter.getEvents().size()); + emitter.verifyEmitted("sys/fs/max", 2); + emitter.verifyEmitted("sys/fs/used", 2); + emitter.verifyEmitted("sys/fs/files/count", 2); + emitter.verifyEmitted("sys/fs/files/free", 2); + Map userDims1 = ImmutableMap.of( + "fsDevName", + "/dev/disk1", + "fsDirName", + "/System/Volumes/boot1" + ); + List metricValues1 = emitter.getMetricValues("sys/fs/max", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(300L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/fs/used", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(100L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/fs/files/count", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(1000L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/fs/files/free", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(700L, metricValues1.get(0)); + + Map userDims2 = ImmutableMap.of( + "fsDevName", + "/dev/disk2", + "fsDirName", + "/System/Volumes/boot2" + ); + List metricValues2 = emitter.getMetricValues("sys/fs/max", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(400L, metricValues2.get(0)); + metricValues2 = emitter.getMetricValues("sys/fs/used", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(80L, metricValues2.get(0)); + metricValues2 = emitter.getMetricValues("sys/fs/files/count", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(800L, metricValues2.get(0)); + metricValues2 = emitter.getMetricValues("sys/fs/files/free", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(600L, metricValues2.get(0)); + m.stop(); + } + + @Test + public void testDiskStats() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + HWDiskStore disk1 = Mockito.mock(HWDiskStore.class); + HWDiskStore disk2 = Mockito.mock(HWDiskStore.class); + Mockito.when(disk1.getReadBytes()).thenReturn(300L); + Mockito.when(disk1.getReads()).thenReturn(200L); + Mockito.when(disk1.getWriteBytes()).thenReturn(400L); + Mockito.when(disk1.getWrites()).thenReturn(500L); + Mockito.when(disk1.getCurrentQueueLength()).thenReturn(100L); + Mockito.when(disk1.getTransferTime()).thenReturn(150L); + Mockito.when(disk1.getName()).thenReturn("disk1"); + Mockito.when(disk2.getReadBytes()).thenReturn(2000L); + Mockito.when(disk2.getReads()).thenReturn(3000L); + Mockito.when(disk2.getWriteBytes()).thenReturn(1000L); + Mockito.when(disk2.getWrites()).thenReturn(4000L); + Mockito.when(disk2.getCurrentQueueLength()).thenReturn(750L); + Mockito.when(disk2.getTransferTime()).thenReturn(800L); + Mockito.when(disk2.getName()).thenReturn("disk2"); + List hwDiskStores = ImmutableList.of(disk1, disk2); + Mockito.when(hal.getDiskStores()).thenReturn(hwDiskStores); + + OshiSysMonitor m = new OshiSysMonitor(si); + m.start(); + m.monitorDiskStats(emitter); + Assert.assertEquals(0, emitter.getEvents().size()); + + Mockito.when(disk1.getReadBytes()).thenReturn(400L); + Mockito.when(disk1.getReads()).thenReturn(220L); + Mockito.when(disk1.getWriteBytes()).thenReturn(600L); + Mockito.when(disk1.getWrites()).thenReturn(580L); + Mockito.when(disk1.getCurrentQueueLength()).thenReturn(300L); + Mockito.when(disk1.getTransferTime()).thenReturn(250L); + Mockito.when(disk2.getReadBytes()).thenReturn(4500L); + Mockito.when(disk2.getReads()).thenReturn(3500L); + Mockito.when(disk2.getWriteBytes()).thenReturn(2300L); + Mockito.when(disk2.getWrites()).thenReturn(5000L); + Mockito.when(disk2.getCurrentQueueLength()).thenReturn(900L); + Mockito.when(disk2.getTransferTime()).thenReturn(1100L); + + m.monitorDiskStats(emitter); + Assert.assertEquals(12, emitter.getEvents().size()); + + Map userDims1 = ImmutableMap.of( + "diskName", + "disk1" + ); + List metricValues1 = emitter.getMetricValues("sys/disk/read/size", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(100L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/disk/read/count", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(20L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/disk/write/size", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(200L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/disk/write/count", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(80L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/disk/queue", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(200L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/disk/transferTime", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(100L, metricValues1.get(0)); + + Map userDims2 = ImmutableMap.of( + "diskName", + "disk2" + ); + List metricValues2 = emitter.getMetricValues("sys/disk/read/size", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(2500L, metricValues2.get(0)); + metricValues2 = emitter.getMetricValues("sys/disk/read/count", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(500L, metricValues2.get(0)); + metricValues2 = emitter.getMetricValues("sys/disk/write/size", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(1300L, metricValues2.get(0)); + metricValues2 = emitter.getMetricValues("sys/disk/write/count", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(1000L, metricValues2.get(0)); + metricValues2 = emitter.getMetricValues("sys/disk/queue", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(150L, metricValues2.get(0)); + metricValues2 = emitter.getMetricValues("sys/disk/transferTime", userDims2); + Assert.assertEquals(1, metricValues2.size()); + Assert.assertEquals(300L, metricValues2.get(0)); + + m.stop(); + } + + @Test + public void testNetStats() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + NetworkIF net1 = Mockito.mock(NetworkIF.class); + Mockito.when(net1.getBytesRecv()).thenReturn(300L); + Mockito.when(net1.getPacketsRecv()).thenReturn(200L); + Mockito.when(net1.getInErrors()).thenReturn(400L); + Mockito.when(net1.getInDrops()).thenReturn(500L); + Mockito.when(net1.getBytesSent()).thenReturn(100L); + Mockito.when(net1.getPacketsSent()).thenReturn(150L); + Mockito.when(net1.getOutErrors()).thenReturn(200L); + Mockito.when(net1.getCollisions()).thenReturn(20L); + Mockito.when(net1.getName()).thenReturn("Wifi"); + Mockito.when(net1.getIPv4addr()).thenReturn(new String[]{"123.456.7.8", "0.0.0.0", "192.1.2.3"}); + Mockito.when(net1.getMacaddr()).thenReturn("ha:rd:wa:re:add"); + + List networkIFS = ImmutableList.of(net1); + Mockito.when(hal.getNetworkIFs()).thenReturn(networkIFS); + + OshiSysMonitor m = new OshiSysMonitor(si); + m.start(); + m.monitorNetStats(emitter); + Assert.assertEquals(0, emitter.getEvents().size()); + + Mockito.when(net1.getBytesRecv()).thenReturn(400L); + Mockito.when(net1.getPacketsRecv()).thenReturn(220L); + Mockito.when(net1.getInErrors()).thenReturn(600L); + Mockito.when(net1.getInDrops()).thenReturn(580L); + Mockito.when(net1.getBytesSent()).thenReturn(300L); + Mockito.when(net1.getPacketsSent()).thenReturn(250L); + Mockito.when(net1.getOutErrors()).thenReturn(330L); + Mockito.when(net1.getCollisions()).thenReturn(240L); + + + m.monitorNetStats(emitter); + Assert.assertEquals(16, emitter.getEvents().size()); // 8 * 2 whitelisted ips + + Map userDims1 = ImmutableMap.of( + "netName", + "Wifi", + "netAddress", + "123.456.7.8", + "netHwaddr", + "ha:rd:wa:re:add" + ); + Map userDims2 = ImmutableMap.of( + "netName", + "Wifi", + "netAddress", + "192.1.2.3", + "netHwaddr", + "ha:rd:wa:re:add" + ); + List metricValues1 = emitter.getMetricValues("sys/net/read/size", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(100L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/read/packets", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(20L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/read/errors", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(200L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/read/dropped", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(80L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/write/size", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(200L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/write/packets", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(100L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/write/errors", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(130L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/write/collisions", userDims1); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(220L, metricValues1.get(0)); + + metricValues1 = emitter.getMetricValues("sys/net/read/size", userDims2); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(100L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/read/packets", userDims2); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(20L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/read/errors", userDims2); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(200L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/read/dropped", userDims2); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(80L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/write/size", userDims2); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(200L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/write/packets", userDims2); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(100L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/write/errors", userDims2); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(130L, metricValues1.get(0)); + metricValues1 = emitter.getMetricValues("sys/net/write/collisions", userDims2); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(220L, metricValues1.get(0)); + m.stop(); + } + + @Test + public void testCpuStats() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + CentralProcessor processor = Mockito.mock(CentralProcessor.class); + long[][] procTicks = new long[][]{ + {1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L}, + {2L, 4L, 6L, 8L, 10L, 12L, 14L, 16L}, + }; + Mockito.when(processor.getProcessorCpuLoadTicks()).thenReturn(procTicks); + Mockito.when(hal.getProcessor()).thenReturn(processor); + + OshiSysMonitor m = new OshiSysMonitor(si); + m.start(); + m.monitorCpuStats(emitter); + Assert.assertEquals(0, emitter.getEvents().size()); + + long[][] procTicks2 = new long[][]{ + {4L, 5L, 6L, 8L, 9L, 7L, 10L, 12L}, // Δtick1 {3,3,3,4,4,1,3,4} _total = 25, emitted percentage + {5L, 8L, 8L, 10L, 15L, 14L, 18L, 22L}, // Δtick2 {3,4,2,2,5,2,4,6} _total = 28 + }; + Mockito.when(processor.getProcessorCpuLoadTicks()).thenReturn(procTicks2); + + m.monitorCpuStats(emitter); + m.stop(); + Assert.assertEquals(16, emitter.getEvents().size()); // 8 ticktype * 2 processors + + Map userDims = new HashMap(); + userDims.put("cpuName", "0"); + userDims.put("cpuTime", "user"); + List metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(12L, metricValues1.get(0)); + userDims.replace("cpuTime", "nice"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(12L, metricValues1.get(0)); + userDims.replace("cpuTime", "sys"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(12L, metricValues1.get(0)); + userDims.replace("cpuTime", "idle"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(16L, metricValues1.get(0)); + userDims.replace("cpuTime", "wait"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(16L, metricValues1.get(0)); + userDims.replace("cpuTime", "irq"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(4L, metricValues1.get(0)); + userDims.replace("cpuTime", "softIrq"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(12L, metricValues1.get(0)); + userDims.replace("cpuTime", "stolen"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(16L, metricValues1.get(0)); + + userDims.replace("cpuName", "1"); + userDims.replace("cpuTime", "user"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(10L, metricValues1.get(0)); + userDims.replace("cpuTime", "nice"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(14L, metricValues1.get(0)); + userDims.replace("cpuTime", "sys"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(7L, metricValues1.get(0)); + userDims.replace("cpuTime", "idle"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(7L, metricValues1.get(0)); + userDims.replace("cpuTime", "wait"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(17L, metricValues1.get(0)); + userDims.replace("cpuTime", "irq"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(7L, metricValues1.get(0)); + userDims.replace("cpuTime", "softIrq"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(14L, metricValues1.get(0)); + userDims.replace("cpuTime", "stolen"); + metricValues1 = emitter.getMetricValues("sys/cpu", userDims); + Assert.assertEquals(1, metricValues1.size()); + Assert.assertEquals(21L, metricValues1.get(0)); + + } + + @Test + public void testSysStats() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + + Mockito.when(os.getSystemUptime()).thenReturn(4000L); + CentralProcessor processor = Mockito.mock(CentralProcessor.class); + double[] la = new double[]{2.31, 4.31, 5.31}; + Mockito.when(processor.getSystemLoadAverage(3)).thenReturn(la); + Mockito.when(hal.getProcessor()).thenReturn(processor); + + OshiSysMonitor m = new OshiSysMonitor(si); + m.start(); + m.monitorSysStats(emitter); + Assert.assertEquals(4, emitter.getEvents().size()); + m.stop(); + emitter.verifyEmitted("sys/uptime", 1); + emitter.verifyEmitted("sys/la/1", 1); + emitter.verifyEmitted("sys/la/5", 1); + emitter.verifyEmitted("sys/la/15", 1); + emitter.verifyValue("sys/uptime", 4000L); + emitter.verifyValue("sys/la/1", 2.31); + emitter.verifyValue("sys/la/5", 4.31); + emitter.verifyValue("sys/la/15", 5.31); + + } + + @Test + public void testTcpStats() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + InternetProtocolStats.TcpStats tcpv4 = Mockito.mock(InternetProtocolStats.TcpStats.class); + InternetProtocolStats ipstats = Mockito.mock(InternetProtocolStats.class); + Mockito.when(tcpv4.getConnectionsActive()).thenReturn(10L); + Mockito.when(tcpv4.getConnectionsPassive()).thenReturn(20L); + Mockito.when(tcpv4.getConnectionFailures()).thenReturn(5L); + Mockito.when(tcpv4.getConnectionsReset()).thenReturn(7L); + Mockito.when(tcpv4.getSegmentsReceived()).thenReturn(200L); + Mockito.when(tcpv4.getInErrors()).thenReturn(3L); + Mockito.when(tcpv4.getSegmentsSent()).thenReturn(300L); + Mockito.when(tcpv4.getOutResets()).thenReturn(4L); + Mockito.when(tcpv4.getSegmentsRetransmitted()).thenReturn(8L); + Mockito.when(ipstats.getTCPv4Stats()).thenReturn(tcpv4); + Mockito.when(os.getInternetProtocolStats()).thenReturn(ipstats); + + OshiSysMonitor m = new OshiSysMonitor(si); + m.start(); + m.monitorTcpStats(emitter); + + Assert.assertEquals(0, emitter.getEvents().size()); + Mockito.when(tcpv4.getConnectionsActive()).thenReturn(20L); + Mockito.when(tcpv4.getConnectionsPassive()).thenReturn(25L); + Mockito.when(tcpv4.getConnectionFailures()).thenReturn(8L); + Mockito.when(tcpv4.getConnectionsReset()).thenReturn(14L); + Mockito.when(tcpv4.getSegmentsReceived()).thenReturn(350L); + Mockito.when(tcpv4.getInErrors()).thenReturn(4L); + Mockito.when(tcpv4.getSegmentsSent()).thenReturn(500L); + Mockito.when(tcpv4.getOutResets()).thenReturn(7L); + Mockito.when(tcpv4.getSegmentsRetransmitted()).thenReturn(8L); + m.monitorTcpStats(emitter); + m.stop(); + Assert.assertEquals(9, emitter.getEvents().size()); + emitter.verifyValue("sys/tcpv4/activeOpens", 10L); + emitter.verifyValue("sys/tcpv4/passiveOpens", 5L); + emitter.verifyValue("sys/tcpv4/attemptFails", 3L); + emitter.verifyValue("sys/tcpv4/estabResets", 7L); + emitter.verifyValue("sys/tcpv4/in/segs", 150L); + emitter.verifyValue("sys/tcpv4/in/errs", 1L); + emitter.verifyValue("sys/tcpv4/out/segs", 200L); + emitter.verifyValue("sys/tcpv4/out/rsts", 3L); + emitter.verifyValue("sys/tcpv4/retrans/segs", 0L); + + } + + private void checkEvents(List events, String expectedFeed) + { + Assert.assertFalse("no events emitted", events.isEmpty()); + for (Event e : events) { + if (!expectedFeed.equals(e.getFeed())) { + String message = StringUtils.format("\"feed\" in event: %s", e.toMap().toString()); + Assert.assertEquals(message, expectedFeed, e.getFeed()); + } + } + } + + +} diff --git a/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java b/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java index f060ec51794..46c0fc90d89 100644 --- a/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java +++ b/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java @@ -46,7 +46,9 @@ import org.apache.druid.java.util.metrics.JvmMonitor; import org.apache.druid.java.util.metrics.JvmThreadsMonitor; import org.apache.druid.java.util.metrics.Monitor; import org.apache.druid.java.util.metrics.MonitorScheduler; +import org.apache.druid.java.util.metrics.NoopOshiSysMonitor; import org.apache.druid.java.util.metrics.NoopSysMonitor; +import org.apache.druid.java.util.metrics.OshiSysMonitor; import org.apache.druid.java.util.metrics.SysMonitor; import org.apache.druid.query.ExecutorServiceMonitor; @@ -192,4 +194,19 @@ public class MetricsModule implements Module return new SysMonitor(dimensions); } } + + @Provides + @ManageLifecycle + public OshiSysMonitor getOshiSysMonitor(DataSourceTaskIdHolder dataSourceTaskIdHolder, @Self Set nodeRoles) + { + if (nodeRoles.contains(NodeRole.PEON)) { + return new NoopOshiSysMonitor(); + } else { + Map dimensions = MonitorsConfig.mapOfDatasourceAndTaskID( + dataSourceTaskIdHolder.getDataSource(), + dataSourceTaskIdHolder.getTaskId() + ); + return new OshiSysMonitor(dimensions); + } + } } diff --git a/server/src/test/java/org/apache/druid/server/metrics/MetricsModuleTest.java b/server/src/test/java/org/apache/druid/server/metrics/MetricsModuleTest.java index 473b549b51d..74899716862 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/MetricsModuleTest.java +++ b/server/src/test/java/org/apache/druid/server/metrics/MetricsModuleTest.java @@ -44,7 +44,9 @@ import org.apache.druid.java.util.emitter.service.ServiceEventBuilder; import org.apache.druid.java.util.metrics.BasicMonitorScheduler; import org.apache.druid.java.util.metrics.ClockDriftSafeMonitorScheduler; import org.apache.druid.java.util.metrics.MonitorScheduler; +import org.apache.druid.java.util.metrics.NoopOshiSysMonitor; import org.apache.druid.java.util.metrics.NoopSysMonitor; +import org.apache.druid.java.util.metrics.OshiSysMonitor; import org.apache.druid.java.util.metrics.SysMonitor; import org.apache.druid.server.DruidNode; import org.hamcrest.CoreMatchers; @@ -198,6 +200,30 @@ public class MetricsModuleTest Assert.assertFalse(sysMonitor instanceof NoopSysMonitor); Mockito.verify(emitter, Mockito.atLeastOnce()).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); } + @Test + public void testGetOshiSysMonitorViaInjector() + { + + final Injector injector = createInjector(new Properties(), ImmutableSet.of(NodeRole.PEON)); + final OshiSysMonitor sysMonitor = injector.getInstance(OshiSysMonitor.class); + final ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); + sysMonitor.doMonitor(emitter); + + Assert.assertTrue(sysMonitor instanceof NoopOshiSysMonitor); + Mockito.verify(emitter, Mockito.never()).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); + } + @Test + public void testGetOshiSysMonitorWhenNull() + { + + Injector injector = createInjector(new Properties(), ImmutableSet.of()); + final OshiSysMonitor sysMonitor = injector.getInstance(OshiSysMonitor.class); + final ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); + sysMonitor.doMonitor(emitter); + + Assert.assertFalse(sysMonitor instanceof NoopOshiSysMonitor); + Mockito.verify(emitter, Mockito.atLeastOnce()).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); + } private static Injector createInjector(Properties properties, ImmutableSet nodeRoles) { From 92a7febacbe71f13cdfc2e8b976cadff57606a0f Mon Sep 17 00:00:00 2001 From: Rishabh Singh <6513075+findingrish@users.noreply.github.com> Date: Wed, 21 Jun 2023 10:42:26 +0530 Subject: [PATCH 11/74] Revert "Add method to authorize native query using authentication result (#14376)" (#14452) This reverts commit 8b212e73d75e08db718a1121da0f201fff723cf2. --- .../apache/druid/server/QueryLifecycle.java | 16 +---- .../druid/server/QueryLifecycleTest.java | 70 +++++-------------- 2 files changed, 20 insertions(+), 66 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java index d47e76a666d..8f0deafac2b 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java @@ -220,18 +220,6 @@ public class QueryLifecycle * @return authorization result */ public Access authorize(HttpServletRequest req) - { - return authorize(AuthorizationUtils.authenticationResultFromRequest(req)); - } - - /** - * Authorize the query using the authentication result. - * Will return an Access object denoting whether the query is authorized or not. - * - * @param authenticationResult authentication result indicating identity of the requester - * @return authorization result of requester - */ - public Access authorize(AuthenticationResult authenticationResult) { transition(State.INITIALIZED, State.AUTHORIZING); final Iterable resourcesToAuthorize = Iterables.concat( @@ -245,9 +233,9 @@ public class QueryLifecycle ) ); return doAuthorize( - authenticationResult, + AuthorizationUtils.authenticationResultFromRequest(req), AuthorizationUtils.authorizeAllResourceActions( - authenticationResult, + req, resourcesToAuthorize, authorizerMapper ) diff --git a/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java b/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java index 385c594b997..578661ea768 100644 --- a/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java @@ -188,15 +188,15 @@ public class QueryLifecycleTest EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) - .andReturn(Access.OK).times(2); + .andReturn(Access.OK); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("foo", ResourceType.QUERY_CONTEXT), Action.WRITE)) - .andReturn(Access.OK).times(2); + .andReturn(Access.OK); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("baz", ResourceType.QUERY_CONTEXT), Action.WRITE)) - .andReturn(Access.OK).times(2); + .andReturn(Access.OK); EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject())) .andReturn(toolChest) - .times(2); + .once(); replayAll(); @@ -223,10 +223,6 @@ public class QueryLifecycleTest ); Assert.assertTrue(lifecycle.authorize(mockRequest()).isAllowed()); - - lifecycle = createLifecycle(authConfig); - lifecycle.initialize(query); - Assert.assertTrue(lifecycle.authorize(authenticationResult).isAllowed()); } @Test @@ -236,15 +232,13 @@ public class QueryLifecycleTest EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) - .andReturn(Access.OK) - .times(2); + .andReturn(Access.OK); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("foo", ResourceType.QUERY_CONTEXT), Action.WRITE)) - .andReturn(Access.DENIED) - .times(2); + .andReturn(Access.DENIED); EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject())) .andReturn(toolChest) - .times(2); + .once(); replayAll(); @@ -261,10 +255,6 @@ public class QueryLifecycleTest QueryLifecycle lifecycle = createLifecycle(authConfig); lifecycle.initialize(query); Assert.assertFalse(lifecycle.authorize(mockRequest()).isAllowed()); - - lifecycle = createLifecycle(authConfig); - lifecycle.initialize(query); - Assert.assertFalse(lifecycle.authorize(authenticationResult).isAllowed()); } @Test @@ -274,12 +264,11 @@ public class QueryLifecycleTest EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) - .andReturn(Access.OK) - .times(2); + .andReturn(Access.OK); EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject())) .andReturn(toolChest) - .times(2); + .once(); replayAll(); @@ -307,10 +296,6 @@ public class QueryLifecycleTest ); Assert.assertTrue(lifecycle.authorize(mockRequest()).isAllowed()); - - lifecycle = createLifecycle(authConfig); - lifecycle.initialize(query); - Assert.assertTrue(lifecycle.authorize(authenticationResult).isAllowed()); } @Test @@ -320,12 +305,11 @@ public class QueryLifecycleTest EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) - .andReturn(Access.OK) - .times(2); + .andReturn(Access.OK); EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject())) .andReturn(toolChest) - .times(2); + .once(); replayAll(); @@ -354,10 +338,6 @@ public class QueryLifecycleTest ); Assert.assertTrue(lifecycle.authorize(mockRequest()).isAllowed()); - - lifecycle = createLifecycle(authConfig); - lifecycle.initialize(query); - Assert.assertTrue(lifecycle.authorize(authenticationResult).isAllowed()); } @Test @@ -367,15 +347,13 @@ public class QueryLifecycleTest EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) - .andReturn(Access.OK) - .times(2); + .andReturn(Access.OK); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("foo", ResourceType.QUERY_CONTEXT), Action.WRITE)) - .andReturn(Access.DENIED) - .times(2); + .andReturn(Access.DENIED); EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject())) .andReturn(toolChest) - .times(2); + .once(); replayAll(); @@ -395,10 +373,6 @@ public class QueryLifecycleTest QueryLifecycle lifecycle = createLifecycle(authConfig); lifecycle.initialize(query); Assert.assertFalse(lifecycle.authorize(mockRequest()).isAllowed()); - - lifecycle = createLifecycle(authConfig); - lifecycle.initialize(query); - Assert.assertFalse(lifecycle.authorize(authenticationResult).isAllowed()); } @Test @@ -408,18 +382,14 @@ public class QueryLifecycleTest EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("fake", ResourceType.DATASOURCE), Action.READ)) - .andReturn(Access.OK) - .times(2); + .andReturn(Access.OK); EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("foo", ResourceType.QUERY_CONTEXT), Action.WRITE)) - .andReturn(Access.OK) - .times(2); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("baz", ResourceType.QUERY_CONTEXT), Action.WRITE)) - .andReturn(Access.OK) - .times(2); + .andReturn(Access.OK); + EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("baz", ResourceType.QUERY_CONTEXT), Action.WRITE)).andReturn(Access.OK); EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject())) .andReturn(toolChest) - .times(2); + .once(); replayAll(); @@ -438,10 +408,6 @@ public class QueryLifecycleTest Assert.assertTrue(revisedContext.containsKey("queryId")); Assert.assertTrue(lifecycle.authorize(mockRequest()).isAllowed()); - - lifecycle = createLifecycle(authConfig); - lifecycle.initialize(query); - Assert.assertTrue(lifecycle.authorize(mockRequest()).isAllowed()); } private HttpServletRequest mockRequest() From 1a9aefbb0f6dfe4eff752bf78bf2df221b5944a5 Mon Sep 17 00:00:00 2001 From: Sergio Ferragut Date: Wed, 21 Jun 2023 09:11:00 -0700 Subject: [PATCH 12/74] Move from Jupyter notebook to Jupyter Lab and introduce a notebook folder structure (#14419) --- .../quickstart/jupyter-notebooks/Dockerfile | 35 +- .../docker-jupyter/docker-compose-local.yaml | 14 +- .../docker-jupyter/docker-compose.yaml | 14 +- .../jupyter-notebooks/kafka-tutorial.ipynb | 788 ------------ .../01-introduction/00-START-HERE.ipynb} | 12 +- .../01-druidapi-package-intro.ipynb} | 18 +- .../01-streaming-from-kafka.ipynb | 537 ++++++++ .../notebooks/02-ingestion/DruidDataDriver.py | 1133 +++++++++++++++++ .../02-ingestion}/kafka_docker_config.json | 0 .../00-using-sql-with-druidapi.ipynb} | 2 +- .../04-api/00-getting-started.ipynb} | 2 +- 11 files changed, 1713 insertions(+), 842 deletions(-) delete mode 100644 examples/quickstart/jupyter-notebooks/kafka-tutorial.ipynb rename examples/quickstart/jupyter-notebooks/{0-START-HERE.ipynb => notebooks/01-introduction/00-START-HERE.ipynb} (91%) rename examples/quickstart/jupyter-notebooks/{Python_API_Tutorial.ipynb => notebooks/01-introduction/01-druidapi-package-intro.ipynb} (99%) create mode 100644 examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb create mode 100644 examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/DruidDataDriver.py rename examples/quickstart/jupyter-notebooks/{docker-jupyter => notebooks/02-ingestion}/kafka_docker_config.json (100%) rename examples/quickstart/jupyter-notebooks/{sql-tutorial.ipynb => notebooks/03-query/00-using-sql-with-druidapi.ipynb} (99%) rename examples/quickstart/jupyter-notebooks/{api-tutorial.ipynb => notebooks/04-api/00-getting-started.ipynb} (99%) diff --git a/examples/quickstart/jupyter-notebooks/Dockerfile b/examples/quickstart/jupyter-notebooks/Dockerfile index 492a4da9c1e..52e10175c21 100644 --- a/examples/quickstart/jupyter-notebooks/Dockerfile +++ b/examples/quickstart/jupyter-notebooks/Dockerfile @@ -31,13 +31,13 @@ FROM jupyter/base-notebook WORKDIR /home/jovyan # Install required Python packages -RUN pip install requests -RUN pip install pandas -RUN pip install numpy -RUN pip install seaborn -RUN pip install bokeh -RUN pip install kafka-python -RUN pip install sortedcontainers +RUN pip install requests \ + pip install pandas \ + pip install numpy \ + pip install seaborn \ + pip install bokeh \ + pip install kafka-python \ + pip install sortedcontainers # Install druidapi client from apache/druid # Local install requires sudo privileges @@ -45,21 +45,22 @@ USER root ADD druidapi /home/jovyan/druidapi WORKDIR /home/jovyan/druidapi RUN pip install . -WORKDIR /home/jovyan + + +# WIP -- install DruidDataDriver as a package # Import data generator and configuration file # Change permissions to allow import (requires sudo privileges) -# WIP -- change to apache repo -ADD https://raw.githubusercontent.com/shallada/druid/data-generator/examples/quickstart/jupyter-notebooks/data-generator/DruidDataDriver.py . -ADD docker-jupyter/kafka_docker_config.json . -RUN chmod 664 DruidDataDriver.py -RUN chmod 664 kafka_docker_config.json + +# The Jupyter notebooks themselves are mounted into the image's /home/jovyan/notebooks +# path when running this image. +RUN mkdir -p /home/jovyan/notebooks + +WORKDIR /home/jovyan/notebooks USER jovyan -# Copy the Jupyter notebook tutorials from the -# build directory to the image working directory -COPY ./*ipynb . + # Add location of the data generator to PYTHONPATH -ENV PYTHONPATH "${PYTHONPATH}:/home/jovyan" +ENV PYTHONPATH "${PYTHONPATH}:/home/jovyan/notebooks/02-ingestion" diff --git a/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose-local.yaml b/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose-local.yaml index e0ea20c17be..3d7baef9052 100644 --- a/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose-local.yaml +++ b/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose-local.yaml @@ -45,9 +45,7 @@ services: zookeeper: image: zookeeper:latest container_name: zookeeper - profiles: ["druid-jupyter", "all-services"] - ports: - - "2181:2181" + profiles: ["druid-jupyter", "kafka-jupyter", "all-services"] environment: - ZOO_MY_ID=1 - ALLOW_ANONYMOUS_LOGIN=yes @@ -55,7 +53,7 @@ services: kafka: image: bitnami/kafka:latest container_name: kafka-broker - profiles: ["all-services"] + profiles: ["kafka-jupyter", "all-services"] ports: # To learn about configuring Kafka for access across networks see # https://www.confluent.io/blog/kafka-client-cannot-connect-to-broker-on-aws-on-docker-etc/ @@ -163,11 +161,13 @@ services: context: .. dockerfile: Dockerfile container_name: jupyter - profiles: ["jupyter", "druid-jupyter", "all-services"] + profiles: ["jupyter", "kafka-jupyter", "druid-jupyter", "all-services"] environment: - DOCKER_STACKS_JUPYTER_CMD: "notebook" + JUPYTER_ENABLE_LAB: "yes" + JUPYTER_TOKEN: "docker" + DOCKER_STACKS_JUPYTER_CMD: "lab" NOTEBOOK_ARGS: "--NotebookApp.token=''" ports: - "${JUPYTER_PORT:-8889}:8888" volumes: - - ./notebooks:/home/jovyan/work + - ../notebooks:/home/jovyan/notebooks diff --git a/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose.yaml b/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose.yaml index 9b784fe8de0..e6f2cd95ae9 100644 --- a/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose.yaml +++ b/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose.yaml @@ -45,9 +45,7 @@ services: zookeeper: image: zookeeper:latest container_name: zookeeper - profiles: ["druid-jupyter", "all-services"] - ports: - - "2181:2181" + profiles: ["druid-jupyter", "kafka-jupyter", "all-services"] environment: - ZOO_MY_ID=1 - ALLOW_ANONYMOUS_LOGIN=yes @@ -55,7 +53,7 @@ services: kafka: image: bitnami/kafka:latest container_name: kafka-broker - profiles: ["all-services"] + profiles: ["kafka-jupyter", "all-services"] ports: # To learn about configuring Kafka for access across networks see # https://www.confluent.io/blog/kafka-client-cannot-connect-to-broker-on-aws-on-docker-etc/ @@ -161,11 +159,13 @@ services: jupyter: image: imply/druid-notebook:latest container_name: jupyter - profiles: ["jupyter", "druid-jupyter", "all-services"] + profiles: ["jupyter", "kafka-jupyter", "druid-jupyter", "all-services"] environment: - DOCKER_STACKS_JUPYTER_CMD: "notebook" + JUPYTER_ENABLE_LAB: "yes" + JUPYTER_TOKEN: "docker" + DOCKER_STACKS_JUPYTER_CMD: "lab" NOTEBOOK_ARGS: "--NotebookApp.token=''" ports: - "${JUPYTER_PORT:-8889}:8888" volumes: - - ./notebooks:/home/jovyan/work + - ../notebooks:/home/jovyan/notebooks diff --git a/examples/quickstart/jupyter-notebooks/kafka-tutorial.ipynb b/examples/quickstart/jupyter-notebooks/kafka-tutorial.ipynb deleted file mode 100644 index b25f49ff954..00000000000 --- a/examples/quickstart/jupyter-notebooks/kafka-tutorial.ipynb +++ /dev/null @@ -1,788 +0,0 @@ -{ - "cells": [ - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "Ingest and query data from Apache Kafka\n", - "\n", - "\n", - "\n", - "This tutorial introduces you to streaming ingestion in Apache Druid using the Apache Kafka event streaming platform.\n", - "Follow along to learn how to create and load data into a Kafka topic, start ingesting data from the topic into Druid, and query results over time. This tutorial assumes you have a basic understanding of Druid ingestion, querying, and API requests." - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Table of contents\n", - "\n", - "* [Prerequisites](#Prerequisites)\n", - "* [Load Druid API client](#Load-Druid-API-client)\n", - "* [Create Kafka topic](#Create-Kafka-topic)\n", - "* [Load data into Kafka topic](#Load-data-into-Kafka-topic)\n", - "* [Start Druid ingestion](#Start-Druid-ingestion)\n", - "* [Query Druid datasource and visualize query results](#Query-Druid-datasource-and-visualize-query-results)\n", - "* [Learn more](#Learn-more)" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Prerequisites\n", - "\n", - "This tutorial works with Druid 25.0.0 or later.\n", - "\n", - "Launch this tutorial and all prerequisites using the `all-services` profile of the Docker Compose file for Jupyter-based Druid tutorials. For more information, see [Docker for Jupyter Notebook tutorials](https://druid.apache.org/docs/latest/tutorials/tutorial-jupyter-docker.html).\n", - "\n", - "If you do not use the Docker Compose environment, you need the following:\n", - "* A running Druid instance.\n", - " * Update the `druid_host` variable to point to your Router endpoint. For example, `druid_host = \"http://localhost:8888\"`.\n", - " * Update the `rest_client` variable to point to your Coordinator endpoint. For example, `\"http://localhost:8081\"`.\n", - "* A running Kafka cluster.\n", - " * Update the Kafka bootstrap servers to point to your servers. For example, `bootstrap_servers=[\"localhost:9092\"]`.\n", - "* The following Python packages:\n", - " * `druidapi`, a Python client for Apache Druid\n", - " * `DruidDataDriver`, a data generator\n", - " * `kafka`, a Python client for Apache Kafka\n", - " * `pandas`, `matplotlib`, and `seaborn` for data visualization\n" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Load Druid API client" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "To start the tutorial, run the following cell. It imports the required Python packages and defines a variable for the Druid client, and another for the SQL client used to run SQL commands." - ] - }, - { - "cell_type": "code", - "execution_count": 1, - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "\n", - "\n" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "import druidapi\n", - "import json\n", - "\n", - "# druid_host is the hostname and port for your Druid deployment. \n", - "# In the Docker Compose tutorial environment, this is the Router\n", - "# service running at \"http://router:8888\".\n", - "# If you are not using the Docker Compose environment, edit the `druid_host`.\n", - "\n", - "druid_host = \"http://router:8888\"\n", - "druid_host\n", - "\n", - "druid = druidapi.jupyter_client(druid_host)\n", - "display = druid.display\n", - "sql_client = druid.sql\n", - "\n", - "# Create a rest client for native JSON ingestion for streaming data\n", - "rest_client = druidapi.rest.DruidRestClient(\"http://coordinator:8081\")" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Create Kafka topic" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "This notebook relies on the Python client for the Apache Kafka. Import the Kafka producer and consumer modules, then create a Kafka client. You use the Kafka producer to create and publish records to a new topic named `social_media`." - ] - }, - { - "cell_type": "code", - "execution_count": 2, - "metadata": {}, - "outputs": [], - "source": [ - "from kafka import KafkaProducer\n", - "from kafka import KafkaConsumer\n", - "\n", - "# Kafka runs on kafka:9092 in multi-container tutorial application\n", - "producer = KafkaProducer(bootstrap_servers='kafka:9092')\n", - "topic_name = \"social_media\"" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "Create the `social_media` topic and send a sample event. The `send()` command returns a metadata descriptor for the record." - ] - }, - { - "cell_type": "code", - "execution_count": 3, - "metadata": {}, - "outputs": [ - { - "data": { - "text/plain": [ - "" - ] - }, - "execution_count": 3, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "event = {\n", - " \"__time\": \"2023-01-03T16:40:21.501\",\n", - " \"username\": \"willow\",\n", - " \"post_title\": \"This title is required\",\n", - " \"views\": 15284,\n", - " \"upvotes\": 124,\n", - " \"comments\": 21,\n", - " \"edited\": \"True\"\n", - "}\n", - "\n", - "producer.send(topic_name, json.dumps(event).encode('utf-8'))" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "To verify that the Kafka topic stored the event, create a consumer client to read records from the Kafka cluster, and get the next (only) message:" - ] - }, - { - "cell_type": "code", - "execution_count": 4, - "metadata": {}, - "outputs": [ - { - "name": "stdout", - "output_type": "stream", - "text": [ - "{\"__time\": \"2023-01-03T16:40:21.501\", \"username\": \"willow\", \"post_title\": \"This title is required\", \"views\": 15284, \"upvotes\": 124, \"comments\": 21, \"edited\": \"True\"}\n" - ] - } - ], - "source": [ - "consumer = KafkaConsumer(topic_name, bootstrap_servers=['kafka:9092'], auto_offset_reset='earliest',\n", - " enable_auto_commit=True)\n", - "\n", - "print(next(consumer).value.decode('utf-8'))" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Load data into Kafka topic" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "Instead of manually creating events to send to the Kafka topic, use a data generator to simulate a continuous data stream. This tutorial makes use of Druid Data Driver to simulate a continuous data stream into the `social_media` Kafka topic. To learn more about the Druid Data Driver, see the Druid Summit talk, [Generating Time centric Data for Apache Druid](https://www.youtube.com/watch?v=3zAOeLe3iAo).\n", - "\n", - "In this notebook, you use a background process to continuously load data into the Kafka topic.\n", - "This allows you to keep executing commands in this notebook while data is constantly being streamed into the topic." - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "Run the following cells to load sample data into the `social_media` Kafka topic:" - ] - }, - { - "cell_type": "code", - "execution_count": 5, - "metadata": {}, - "outputs": [], - "source": [ - "import multiprocessing as mp\n", - "from datetime import datetime\n", - "import DruidDataDriver" - ] - }, - { - "cell_type": "code", - "execution_count": 6, - "metadata": {}, - "outputs": [], - "source": [ - "def run_driver():\n", - " DruidDataDriver.simulate(\"kafka_docker_config.json\", None, None, \"REAL\", datetime.now())\n", - " \n", - "mp.set_start_method('fork')\n", - "ps = mp.Process(target=run_driver)\n", - "ps.start()" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Start Druid ingestion" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "Now that you have a new Kafka topic and data being streamed into the topic, you ingest the data into Druid by submitting a Kafka ingestion spec.\n", - "The ingestion spec describes the following:\n", - "* where to source the data to ingest (in `spec > ioConfig`),\n", - "* the datasource to ingest data into (in `spec > dataSchema > dataSource`), and\n", - "* what the data looks like (in `spec > dataSchema > dimensionsSpec`).\n", - "\n", - "Other properties control how Druid aggregates and stores data. For more information, see the Druid documenation:\n", - "* [Apache Kafka ingestion](https://druid.apache.org/docs/latest/development/extensions-core/kafka-ingestion.html)\n", - "* [Ingestion spec reference](https://druid.apache.org/docs/latest/ingestion/ingestion-spec.html)\n", - "\n", - "Run the following cells to define and view the Kafka ingestion spec." - ] - }, - { - "cell_type": "code", - "execution_count": 7, - "metadata": {}, - "outputs": [], - "source": [ - "kafka_ingestion_spec = \"{\\\"type\\\": \\\"kafka\\\",\\\"spec\\\": {\\\"ioConfig\\\": {\\\"type\\\": \\\"kafka\\\",\\\"consumerProperties\\\": {\\\"bootstrap.servers\\\": \\\"kafka:9092\\\"},\\\"topic\\\": \\\"social_media\\\",\\\"inputFormat\\\": {\\\"type\\\": \\\"json\\\"},\\\"useEarliestOffset\\\": true},\\\"tuningConfig\\\": {\\\"type\\\": \\\"kafka\\\"},\\\"dataSchema\\\": {\\\"dataSource\\\": \\\"social_media\\\",\\\"timestampSpec\\\": {\\\"column\\\": \\\"__time\\\",\\\"format\\\": \\\"iso\\\"},\\\"dimensionsSpec\\\": {\\\"dimensions\\\": [\\\"username\\\",\\\"post_title\\\",{\\\"type\\\": \\\"long\\\",\\\"name\\\": \\\"views\\\"},{\\\"type\\\": \\\"long\\\",\\\"name\\\": \\\"upvotes\\\"},{\\\"type\\\": \\\"long\\\",\\\"name\\\": \\\"comments\\\"},\\\"edited\\\"]},\\\"granularitySpec\\\": {\\\"queryGranularity\\\": \\\"none\\\",\\\"rollup\\\": false,\\\"segmentGranularity\\\": \\\"hour\\\"}}}}\"" - ] - }, - { - "cell_type": "code", - "execution_count": 8, - "metadata": {}, - "outputs": [ - { - "name": "stdout", - "output_type": "stream", - "text": [ - "{\n", - " \"type\": \"kafka\",\n", - " \"spec\": {\n", - " \"ioConfig\": {\n", - " \"type\": \"kafka\",\n", - " \"consumerProperties\": {\n", - " \"bootstrap.servers\": \"kafka:9092\"\n", - " },\n", - " \"topic\": \"social_media\",\n", - " \"inputFormat\": {\n", - " \"type\": \"json\"\n", - " },\n", - " \"useEarliestOffset\": true\n", - " },\n", - " \"tuningConfig\": {\n", - " \"type\": \"kafka\"\n", - " },\n", - " \"dataSchema\": {\n", - " \"dataSource\": \"social_media\",\n", - " \"timestampSpec\": {\n", - " \"column\": \"__time\",\n", - " \"format\": \"iso\"\n", - " },\n", - " \"dimensionsSpec\": {\n", - " \"dimensions\": [\n", - " \"username\",\n", - " \"post_title\",\n", - " {\n", - " \"type\": \"long\",\n", - " \"name\": \"views\"\n", - " },\n", - " {\n", - " \"type\": \"long\",\n", - " \"name\": \"upvotes\"\n", - " },\n", - " {\n", - " \"type\": \"long\",\n", - " \"name\": \"comments\"\n", - " },\n", - " \"edited\"\n", - " ]\n", - " },\n", - " \"granularitySpec\": {\n", - " \"queryGranularity\": \"none\",\n", - " \"rollup\": false,\n", - " \"segmentGranularity\": \"hour\"\n", - " }\n", - " }\n", - " }\n", - "}\n" - ] - } - ], - "source": [ - "print(json.dumps(json.loads(kafka_ingestion_spec), indent=4))" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "Send the spec to Druid to start the streaming ingestion from Kafka:" - ] - }, - { - "cell_type": "code", - "execution_count": 9, - "metadata": {}, - "outputs": [ - { - "data": { - "text/plain": [ - "" - ] - }, - "execution_count": 9, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "headers = {\n", - " 'Content-Type': 'application/json'\n", - "}\n", - "\n", - "rest_client.post(\"/druid/indexer/v1/supervisor\", kafka_ingestion_spec, headers=headers)" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "A `200` response indicates that the request was successful. You can view the running ingestion task and the new datasource in the web console at http://localhost:8888/unified-console.html." - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Query Druid datasource and visualize query results" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "You can now query the new datasource called `social_media`. In this section, you also visualize query results using the Matplotlib and Seaborn visualization libraries. Run the following cell import these packages." - ] - }, - { - "cell_type": "code", - "execution_count": 10, - "metadata": {}, - "outputs": [], - "source": [ - "import pandas as pd\n", - "import matplotlib\n", - "import matplotlib.pyplot as plt\n", - "import seaborn as sns" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "Run a simple query to view a subset of rows from the new datasource:" - ] - }, - { - "cell_type": "code", - "execution_count": 11, - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "

\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "
__timeusernamepost_titleviewsupvotescommentsedited
2023-01-03T16:40:21.501ZwillowThis title is required1528412421True
2023-05-02T23:34:54.451Zgus3y4hkmd1!'Er4;40319315False
2023-05-02T23:34:55.454Zmiam62u53:D9s2bOvnY_VM9vjtZ'MyDLvQ7_xGodAP:ZNTXM6cFAt,_jrxBVBeRILLvAF9Z!jM9YNN;3ErV5eGbE_TFQS16060848True
2023-05-02T23:34:55.455ZjojorAmeAJrjs;FBj:zy2MwoGh_P_SowlLTfp6zhX55xqogH.,1DC2xY_x2T;M_Vcu3QWaz650u;Roa14313657False
2023-05-02T23:34:56.456Zwillow3bHB,iJdE;sedTDA,1dKGDAZL!qdsvO_tv.4Jrq7fa.KWcHPD'TB_5nnvsf9EgtnN8tGeeA0MjKc30iubJ:D'l7pHNihWpFz8K'46q!vJs42371123True
" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "sql = '''\n", - "SELECT * FROM social_media LIMIT 5\n", - "'''\n", - "display.sql(sql)" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "In this social media scenario, each incoming event represents a post on social media, for which you collect the timestamp, username, and post metadata. You are interested in analyzing the total number of upvotes for all posts, compared between users. Preview this data with the following query:" - ] - }, - { - "cell_type": "code", - "execution_count": 12, - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "
\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "
num_poststotal_upvotesusername
15510985willow
16111223gus
16411456leon
17312098jojo
17612175mia
17711998milton
18513256miette
18813360rocket
" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "sql = '''\n", - "SELECT\n", - " COUNT(post_title) as num_posts,\n", - " SUM(upvotes) as total_upvotes,\n", - " username\n", - "FROM social_media\n", - "GROUP BY username\n", - "ORDER BY num_posts\n", - "'''\n", - "\n", - "response = sql_client.sql_query(sql)\n", - "response.show()" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "Visualize the total number of upvotes per user using a line plot. You sort the results by username before plotting because the order of users may vary as new results arrive." - ] - }, - { - "cell_type": "code", - "execution_count": 13, - "metadata": {}, - "outputs": [ - { - "data": { - "image/png": "iVBORw0KGgoAAAANSUhEUgAAAk0AAAHMCAYAAADI/py4AAAAOXRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjcuMCwgaHR0cHM6Ly9tYXRwbG90bGliLm9yZy88F64QAAAACXBIWXMAAA9hAAAPYQGoP6dpAACRN0lEQVR4nOzdd3iTZfcH8O+T7j3ppNAySwdtAQtF9p7KEEVl+JMhvjJERJYyVBRRnCjI60AcrwqyQaBskLJbSiktUArdmzZd6UjO7480sWE2Je2TpOdzXb20z/M0OUlpcnKf+z63QEQExhhjjDH2UBKxA2CMMcYYMwScNDHGGGOM1QEnTYwxxhhjdcBJE2OMMcZYHXDSxBhjjDFWB5w0McYYY4zVASdNjDHGGGN1YCp2AMZCoVAgIyMDdnZ2EARB7HAYY4wxVgdEhOLiYnh5eUEiefhYEidNOpKRkQEfHx+xw2CMMcZYPaSmpqJ58+YPvYaTJh2xs7MDoHzS7e3tRY6GMcYYY3UhlUrh4+Ojfh9/GE6adERVkrO3t+ekiTHGGDMwdZlawxPBGWOMMcbqgJMmxhhjjLE64KSJMcYYY6wOOGlijDHGGKsDTpoYY4wxxuqAkybGGGOMsTrgpIkxxhhjrA44aWKMMcYYqwNOmhhjjDHG6oA7gjPGGGN6TK4gnE0uQE6xDG52lgj3c4aJhDeGFwMnTYwxxpie2heXiRW74pFZJFMf83SwxLKRARgS5CliZE0Tl+cYY4wxPbQvLhOv/nJRI2ECgKwiGV795SL2xWWKFFnTxUkTY4wxpmfkCsKKXfGg+5xTHVuxKx5yxf2uYA2FkybGGGNMz5xNLrhnhKk2ApBZJMPZ5ILGC4px0sQYY4zpm5ziBydM9bmO6QYnTYwxxpiecbOz1Ol1TDc4aWKMMcb0TLifMzwdLPGgxgIClKvowv2cGzOsJo+TJsYYY0zPmEgELBsZcN9zqkRq2cgA7tfUyDhpYowxxvTQkCBPvNK71T3HnW3MsW5CJ+7TJAJOmhhjjDE9pVpBNyzIAx2b2wMAXundihMmkXDSxBhjjOkhWZUcB+OzAQBTe7XCgA4eAICrmcVihtWk8TYqjDHGmB46di0XpZVyeDlYIszHEXdKKwEAVzKKRI6s6eKRJsYYY0wP7YlVbpMyLNgTgiAg0MsBAJCUWwpZlVzM0JosTpoYY4wxPSOrkuPQVWVpblhH5fwld3sLuNiYQ64gJGRxiU4MnDQxxpgekysIUUn52BGTjqikfN5rrIk4mqgszXk7WiHMxxEAIAgCAryUk8G5RCcOntPEGGN6al9cJlbsitfYg8zTwRLLRgbw6ikjt/eysjQ3NMgDgvBvL6ZALwecuJ6HKxlSsUJr0nikiTHG9NC+uEy8+svFezZtzSqS4dVfLmJfXKZIkbGGJquS42BNaW54R83kOFA90sRJkxg4aWKMMT0jVxBW7IrH/QpxqmMrdsVzqc5IHU3MRVlNaS60pjSnokqaEjKlqJYrRIiuaeOkiTHG9MzZ5IJ7RphqIyibHp5NLmi8oFij2XNZtWpOszQHAL4uNrAxN0FFtQI380rFCK9J46SJMcb0TE7xgxOm+lzHDEftVXPDO3rdc14iEdDBkyeDi4WTJsYY0zNudpY6vY4ZjqOJOerSXEhzh/teo57XlM7zmhobJ02MMaZnwv2c4elgiQftXy9AuYou3M+5McNijWB3TUPL4R097ynNqQR6K5OpOB5panScNDHGmJ4xkQhYNjLgvhPBVZaNDICJ5EFpFTNE5ZVyHE7IAQAMD35wSwnVSFN8hhREvBigMXHSxBhjemhIkCc6PqA8839P+nKfJiOkKs01d7J64O8eANq62cHMRIBUVo20O+WNGCHjpIkxxvRQakEZ4tKV5ZdPxoXgi/GhGBOmnBjMq6aM0+6aVXPDgx9cmgMAc1MJ2rnbAeDJ4I2NkybGGNNDv55JgYKAHm1c8Uzn5ng61Buz+rcDAJy4nofc4gqRI2S6VF4px+GrytLcsIeU5lS4yaU4OGlijDE9I6uS449zKQCASREt1cf9XG0Q6uMIuYKw61KGWOGxBnAkMQflVY8uzakEeimv4aSpcXHSxBhjemZ3bCbulFXB29EK/Tu4a5wbHeYNANgWnS5GaKyB7KljaU4lkDfuFQUnTYwxpkeICD+dugUAeLFbi3tWyI0M8YKpRMDl9CLcyCkWIUKma7VLc3fvNfcgHTztIQhAtrQCeSVcqm0snDQxxpgeiUktxOX0IpibSvBcF597zjvbmKNP+2YAeLTJWKhKcz7OVgj2fnRpDgBsLEzh52IDgEt0jYmTJsYY0yObom4DAEZ29IKLrcV9rxlVU6LbHp0BBW/aa/D2xKr2mqtbaU4lgEt0jY6TJsYY0xO5xRXqN9DJ3Vs+8LoBHdxhZ2GK9MJynLvFm/YasrLKahxKUO41NyL43r3mHoYngzc+TpoYY0xP/HEuBZVyBUJ9HNGxueMDr7M0M1EvS+cSnWE7kpALWZUCPs5WCPK21+pna3cGZ42DkybGGNMD1XIFfj2jbDPwsFEmFVWJbs/lTMiq5A0aG2s4ey4rW0cMD/bSqjQH/Js0JeeVolhWpfPY2L04aWKMMT1w8Go2MotkcLExr1Nzw65+zvB2tEKxrFq9XxkzLGWV1XXaa+5BXGwt4OlgCQC4mskrKRsDJ02MMaYHfjqlnAA+PtwHFqYmj7xeIhHwdKhyDszWi1yiM0SHE3Igq1KghbO11qU5Fe7X1Lg4aWKMMZFdyy5G1M18SATgxa6PLs2pqBpdHk3MQUFpZUOFxxrIXlVDy47arZqrLYAngzcqTpoYY0xkm6JuAQAGBXjAy9Gqzj/X1t0OQd72qFYQ9sTytiqGpLTi8UpzKrwHXePipIkxxkQklVWpy2uT6jAB/G6jw5oDALbyKjqDoirNtXSxVic+9aH62evZxaio5gUBDY2TJsYYE9HWC2koq5SjrZstIlq5aP3zI0M8IRGA6JRC3MorbYAIWUNQlea0bWh5N29HKzhYmaFaQbieXaKr8NgDcNLEGGMiUShI3QF8UkTLer15utlZomdb3lbFkOiqNAcAgiDwZPBGxEkTY4yJ5J+kPNzMK4WthSlGd2pe79sZ06lmW5WYdBDxtir67nBCDiqqH780p8LzmhoPJ02MMSYSVZuBZzo3h62Fab1vZ2CAO6zNTXA7vwwXUwp1FB1rKKqtcoY/ZmlOhbdTaTycNDHGmAhSC8rUe45N6Kb9BPDarM1NMSTIAwCwLTrtsWNjDae0ohpHEmtKcx0frzSnohppupophZw3cG5QnDQxxpgIfj2TAiKgZ1tXtHGzfezbU/Vs2h2bicpqxWPfHmsYh2pKc74u1gjwfPzSHAC0amYLSzMJyirluJXPiwEaEidNjDHWyGRVcvxxTrnP3MTHHGVS6d7aFe72Figsq8LRRN5WRV+p+mk9TkPLu5lIBPh78LymxsBJE2OMNbJdlzJwp6wK3o5W6N/BXSe3aSIR8HSocrSJV9Hpp5KKahxNzAWAOu0vqA1eQdc4OGlijLFGRET4qaYD+IRuLWEi0c1oAwCMqkmaDl3NQVEZ73qvbw5dzUZFtQJ+rjY6K82pBHnXTAZP55GmhsRJE2OMNaLo1ELEpUthbirBc0/46PS2A7zs4e9hh0q5AnvjMnV62+zxqfea09GqudpqjzRx24mGw0kTY4w1ok2nbgEAngrxgrONuc5vXzUhfNtFLtHpk5KKahxpoNIcALRzt4OJRMCdsipkFsl0fvtMiZMmxhhrJLnFFdh7OQsAMDnCt0Hu46lQLwgCcPZWAVILyhrkPpj2Dl3NRmW1Aq1cbdDB007nt29pZoK2NasweTJ4wxE1aTp+/DhGjhwJLy8vCIKA7du3a5xfvnw5/P39YWNjAycnJwwYMABnzpzRuKagoAAvvvgi7O3t4ejoiClTpqCkRHP/ndjYWPTs2ROWlpbw8fHB6tWr74ll8+bN8Pf3h6WlJYKDg7F3716dP17GWNP2x7kUVMoVCPVxRHBzhwa5D08HK3RvrdzDbkcMjzbpC1VDy8fda+5hAngyeIMTNWkqLS1FSEgIvv766/ueb9euHdauXYvLly/j5MmT8PX1xaBBg5Cbm6u+5sUXX8SVK1cQGRmJ3bt34/jx45g+fbr6vFQqxaBBg9CyZUtcuHABH3/8MZYvX44NGzaorzl16hSef/55TJkyBdHR0Rg1ahRGjRqFuLi4hnvwjLEmpVquwC+nlW0GJnfXTZuBBxkdptySZWs0b6uiD4plVTh6Tfm+pauGlvfDncEbAekJALRt27aHXlNUVEQA6ODBg0REFB8fTwDo3Llz6mv+/vtvEgSB0tPTiYjom2++IScnJ6qoqFBfs2DBAmrfvr36+2effZaGDx+ucV9du3alV155pc7xq2IrKiqq888wxpqOvbEZ1HLBbur07gGSVVU36H0Vy6qo/dt7qeWC3RSTcqdB74s92vboNGq5YDf1/fgIKRSKBrufqKQ8arlgN3X/8FCD3Ycx0ub922DmNFVWVmLDhg1wcHBASEgIACAqKgqOjo7o0qWL+roBAwZAIpGoy3hRUVHo1asXzM3/nXA5ePBgJCYm4s6dO+prBgwYoHF/gwcPRlRU1APjqaiogFQq1fhijLEHUbUZeD68BSxMTRr0vmwtTDEoQLWtCpfoxLZbtdecDhta3o+qPJdeWI47pZUNdj9Nmd4nTbt374atrS0sLS3x2WefITIyEq6urgCArKwsuLm5aVxvamoKZ2dnZGVlqa9xd9dsHqf6/lHXqM7fz4cffggHBwf1l4+PbpcOM8aMR2JWMU7fLICJRMALXVs0yn2O7qRcRbfrUgaq5LytiliKZVU41gilOQCwtzRDC2drAEB8Jn+Qbwh6nzT17dsXMTExOHXqFIYMGYJnn30WOTnibxGwaNEiFBUVqb9SU1PFDokxpqd+Pn0LADAowB1ejlaNcp8927jC1dYc+aWVOHE999E/wBrEoas5ylVzzWzQ3l33q+buxp3BG5beJ002NjZo06YNunXrhu+//x6mpqb4/vvvAQAeHh73JFDV1dUoKCiAh4eH+prs7GyNa1TfP+oa1fn7sbCwgL29vcYXY4zdTSqrwtaankkTIxp2AnhtpiYSjAzxAgBsi85otPtlmlSluRENuGqutn+TJh5pagh6nzTdTaFQoKKiAgAQERGBwsJCXLhwQX3+8OHDUCgU6Nq1q/qa48ePo6rq3y0FIiMj0b59ezg5OamvOXTokMb9REZGIiIioqEfDmPMyP11IQ1llXK0dbNFRCuXRr3vMTWr6A5cyUKxjLdVaWzFsiocrynNDWvg0pwKr6BrWKImTSUlJYiJiUFMTAwAIDk5GTExMUhJSUFpaSkWL16M06dP4/bt27hw4QJefvllpKenY9y4cQCADh06YMiQIZg2bRrOnj2Lf/75BzNnzsT48ePh5aX8hPXCCy/A3NwcU6ZMwZUrV/DHH3/giy++wBtvvKGOY86cOdi3bx/WrFmDhIQELF++HOfPn8fMmTMb/TlhjBkPhYLwc9RtAMCk7r6NMtJQW5C3PVo3s0FFtQJ/xz14jiZrGAevZqNSrkDrRirNAUCgt3KkKSm3BGWV1Y1yn02JqEnT+fPnERYWhrCwMADAG2+8gbCwMCxduhQmJiZISEjA2LFj0a5dO4wcORL5+fk4ceIEAgMD1bfx66+/wt/fH/3798ewYcPQo0cPjR5MDg4OOHDgAJKTk9G5c2fMmzcPS5cu1ejl1L17d/z222/YsGEDQkJCsGXLFmzfvh1BQUGN92QwxozOyRt5uJlXCjsLU4yp2d6kMQmCgDGdlKNN23kVXaPbE6tMVBtir7kHcbOzRDM7CxABVzOLG+U+mxKBiDuf6YJUKoWDgwOKiop4fhNjDAAw9adzOHg1By9198XypwIf/QMNIO1OGXp8dASCAJxa2A+eDo0zEb2pk8qq0OW9g6iUK7D/9V5o79E4I00A8NKPZ3E0MRfvPR2IiQ20XY8x0eb92+DmNDHGmCFILSjDoQTlQpXGnAB+t+ZO1gj3cwYRsJ0nhDeag/HK0lwbN1u0c7dt1PvmyeANh5MmxhhrAL+cuQ0ioGdbV7Ru1rhvmndTlQa3RafxtiqNZO/lht9r7kF4MnjD4aSJMcZ0TFYlxx/nlL3bJulBeWRosCfMTSW4ll3CTQ8bgVRWhePX8gAAIxpp1VxtqpGmxKxibmyqY5w0McaYju28lIHCsip4O1qhn7/bo3+ggTlYmWFAB2Uc2y7yhPCGplmaa7y5TCo+TtawszBFpVyBGzkljX7/xoyTJsYY0yEiwk+nbgFQzmUykTRuaeZBRtf0bNpxKQNyBZfoGtIe1V5zwY0/ygQAEomADjyvqUFw0sQYYzp0MaUQVzKksDCV4Lku+rMnZe92zeBkbYbc4gr8cyNP7HCMVlF5FU5cVz6/Db3X3MPwdioNg5MmxhjToZ+jbgEARoZ4wcnGXNxgajE3lWBER9W2Klyiayiq0lxbkUpzKjwZvGFw0sQYYzqSW1yBPTWrpibrwQTwu43upFxFty8uC6UV3C26Iah+/2KOMgH/jjRdzZBCweVYneGkiTHGdOT3symokhPCWjgiuLmD2OHcI8zHEb4u1iivkuNAPG+romvK0pxyrzmx5jOptHGzhbmpBMUV1Ui9UyZqLMaEkybGGNOBKrkCv55JAaCfo0yAcluVUTU9m7byKjqdi4zPRpWc0M7dFm1FLM0BgJmJRL3fHZfodIeTJsYY04HI+GxkSWVwtTXH0GAPscN5oNE1SdM/N/KQI5WJHI1xqd3QUh8E1WzeG5fOk8F1hZMmxhjTgU01E8CfD28BC1MTcYN5iJYuNujUwhEKUvaTYrqhT6U5lQCeDK5znDQxxthjSswqxumbBTCRCHihawuxw3mk0Z2UPZu4RKc7qtJce3c70UtzKrwHne5x0sQYY49JNco0KMAdng5W4gZTByOCPWFmIiA+U4rErGKxwzEKe2KVo3b6UpoDgA4e9pAIQF5JBZdidYSTJsYYewxF5VXqERt92GeuLpxszNGnfc22Ktyz6bEVlVXh5A1VQ0v9mc9mZW6CVjWbRfNok25w0sQYY4/hrwtpKK+So527Lbq1chY7nDobUzMhfEdMOvfxeUwH4rPUpbk2bvpRmlPhzuC6xUkTY4zVk0JB+Pn0bQDKUSZB0I995uqir78b7C1NkVkkw+nkfLHDMWj60tDyfnhek25x0sQYY/V08kYekvNKYWdhql7KbygszUzUb/LbeEJ4vRWVVeFkzV5z+jSfSYW3U9EtTpoYY6yeVBPAx3ZuDhsLU3GDqYfRYcpVdH/HZUFWJRc5GsO0Pz4L1QqCv4cd2rjZih3OPVQjTSkFZZDKqkSOxvBx0sQYY/WQWlCGQwk5AICJES1FjqZ+urR0QnMnK5RUVCMyPlvscAzSntia0pwejjIBgKO1ObwdlSs643m06bFx0sQYY/Xwy+nbIAJ6tnVF62b6N8JQFxKJgFGhyrIir6LTXmFZJf6pWTU3TA/nM6kE8LwmneGkiTHGtFReKcfv51IB6O8+c3U1upMyaTp2LRf5JRUiR2NYDlzJVpfm9Dlx5hV0usNJE2OMaWnXpQwUlVehuZMV+vq7iR3OY2ndzBYhzR0gVxB28bYqWlGvmtPT0pxKkGoyeDqPND0uTpoYY0wLRISfaiaAT+zWEiYSw2kz8CCjwrhEpy1DKc0BQGDNxr03ckt4wv9j4qSJMca0cDGlEFcypLAwleDZLj5ih6MTI0O8YCIRcCmtCEm5JWKHYxBUpbkOnvZ6XZoDAA97SzjbmEOuIN425zFx0sQYY1pQtRl4KsQLTjbm4gajI662FujdrhkAYDuPNtXJbnVpTn+2TXkQQRC4yaWOcNLEGGN1lFMsw96aN8vJ3X3FDUbHapfoeFuVh7tTWqs0p+fzmVQCeDK4Tjx20iSXyxETE4M7d+7oIh7GGNNbv59NRZWc0KmFI4K8HcQOR6cGdnCHrYUp0u6U40IKv54/zIH4LMhrSnOt9Lw0p8KdwXVD66Tp9ddfx/fffw9AmTD17t0bnTp1go+PD44eParr+BhjTC9UyRX49YxynzljG2UCACtzEwwJUpaatvK2Kg+1u6ah5Qg9nwBem6o8l5AlhZxHEutN66Rpy5YtCAkJAQDs2rULycnJSEhIwNy5c7FkyRKdB8gYY/ogMj4b2dIKuNpaYGiQ4bxZamNMTYluT2wGr7J6gDullTiVpNzg2FBKcwDg52IDa3MTyKoUuMmT/etN66QpLy8PHh7KTyN79+7FuHHj0K5dO7z88su4fPmyzgNkjDF98NOpWwCA58N9YG5qnNNBu7VygaeDJaSyahxNzBE7HL20/4qyNBfgaQ8/Vxuxw6kziURAB0+eDP64tP7Ld3d3R3x8PORyOfbt24eBAwcCAMrKymBiYqLzABljTGwJWVKcSS6AiUTAC11biB1Og5FIBDwV6gWAS3QPom5oaUClORXuDP74tE6a/u///g/PPvssgoKCIAgCBgwYAAA4c+YM/P39dR4gY4yJbVOUci7T4EB3eDpYiRxNwxoT1hwAcCQxB3dKK0WORr8U1CrN6XsX8PvhtgOPz1TbH1i+fDmCgoKQmpqKcePGwcLCAgBgYmKChQsX6jxAxhgTU1F5FbbVjLpMMvB95uqivYcdAjztEZ8pxZ7LmZjQraXYIekNVWku0MsevgZUmlOpvYKOiCAIht/NvrFpnTQBwDPPPAMAkMlk6mOTJ0/WTUSMMaZH/rqQhvIqOdq726Grn7PY4TSK0WHeiM+UYlt0OidNtah6dBnSBPDa2rrbwlQioKi8CumF5WjuZC12SAZH6/KcXC7He++9B29vb9ja2uLmzZsAgHfeeUfdioAxxoyBQkH4+bSyNDcxomWT+WT+dKgXJAJw4fYd3M4vFTscvWDopTkAsDA1QTt3OwBAHG/eWy9aJ00rV67Exo0bsXr1apib/7uFQFBQEL777judBscYY2I6cSMPyXmlsLMwxeia5fhNgZu9JZ5s4woA2B6dIXI0+sHQS3MqqnlN8TwZvF60Tpo2bdqEDRs24MUXX9RYLRcSEoKEhASdBscYY2LaVNNm4JkuzWFjUa/ZDAZrtHpblTQQcTPEPbGGu2quNp4M/ni0TprS09PRpk2be44rFApUVVXpJCjGGBNbSn4ZDtf0KprYBOf1DA70gJWZCW7llyE6tVDscESVX1KBU0nKveYMtTSnEujN26k8Dq2TpoCAAJw4ceKe41u2bEFYWJhOgmKMMbH9cuY2iIBe7ZoZzP5iumRjYareVmV7dNPu2bT/SjYUBAR526Oli+GW5gCgg6c9BAHIksqQX1IhdjgGR+vx5qVLl2Ly5MlIT0+HQqHA1q1bkZiYiE2bNmH37t0NESNjjDWq8ko5/jiXCgCYHNH0RplURoV5Y1t0OnZdysDbwwOMthP6o+y5rJzXNTzYS+RIHp+thSl8XWyQnFeKKxlS9GrXTOyQDIrWfwFPP/00du3ahYMHD8LGxgZLly7F1atXsWvXLnV3cMYYM2S7LmWgqLwKzZ2s0Ke9m9jhiObJ1i5oZmeBO2VVOHYtV+xwRJFfUoEoA181d7cAntdUb/Wa2dizZ09ERkbqOhbGGBMdEWFjzQTwid1awkTSNNoM3I+piQRPh3jhu5PJ2B6djoEB7mKH1Oj2XcmCgoBgbwe0cDGOvkaBXvbYE5vJ26nUg9YjTa1atUJ+fv49xwsLC9GqVSudBMUYY2K5mHIH8ZlSWJhK8GwXH7HDEd2omlV0kVezUVTe9Bb77DXgveYeRNUZPJ5HmrSmddJ069YtyOXye45XVFQgPb1pTxZkjBm+n04pm1k+HeoFJxvzR1xt/AK97NHO3RaV1Qr8XZNANBV5RliaA/5tO5CcX4rSimqRozEsdS7P7dy5U/3/+/fvh4ODg/p7uVyOQ4cOwdfXV6fBMcZYY8opluHvOGVi0BT2masLQRAwOqw5PtqXgG3R6Rgf3kLskBrNvjhlaa5jcwf4OBtHaQ4AXG0t4G5vgWxpBa5mStHFt2lsD6QLdU6aRo0aBUD5B3T3PnNmZmbw9fXFmjVrdBocY4w1pt/PpqJKTujc0glB3g6P/oEm4ulQL6zen4AzyQVIu1PWZPYsM/S95h4m0MsB2dIcXMngpEkbdS7PKRQKKBQKtGjRAjk5OervFQoFKioqkJiYiBEjRjRkrIwx1mCq5Ar8ekZZmpvUhNsM3I+XoxW6+bkAAHbENI1tVfJKKnD6pvGV5lRUJbq4dJ4Mrg2t5zQlJyfD1dW1IWJhjDHRHLiSjWxpBVxtLTA0yPjeJB/X6E7KCeFbLzaNbVWMtTSnopoMzm0HtFOvTmXHjh3DyJEj0aZNG7Rp0wZPPfXUfbuEM8aYofgp6hYA4IVwnybbxPFhhgZ5wMJUgqTcUsSlG/8brXqvOSMcZQL+HWm6nlOMymqFyNEYDq1fGX755RcMGDAA1tbWmD17NmbPng0rKyv0798fv/32W0PEyBhjDepqphRnkwtgIhHwQlcuzd2PnaWZuk/T1ug0kaNpWLnFFTiTrCzNGeN8JgBo7mQFByszVMkJ17KLxQ7HYGidNK1cuRKrV6/GH3/8oU6a/vjjD6xatQrvvfdeQ8TIGGMNalOUci7TkEAPeDhYihyN/hpd07Np16UMVMuNd3RC1dAyxEhLc4ByUVeAp3K0ifs11Z3WSdPNmzcxcuTIe44/9dRTSE5O1klQjDHWWIrKq9Qb0vIE8Ifr1a4ZXGzMkVdSiRM38sQOp8Hsia3Za86IGlreT6B6OxWeDF5XWidNPj4+OHTo0D3HDx48CB8f7p7LGDMsWy6kobxKjvbudgj346XXD2NmIsHIEOWmtapE09jkFMtwNrkAAIx+QUCgN+9Bpy2t956bN28eZs+ejZiYGHTv3h0A8M8//2Djxo344osvdB4gY4w1FIWC8HPNBPBJ3VtCEJruPnN1NSrMGxtP3cL+K1koqaiGrUW9tjDVW/trVs2F+DgabWlORbWC7mqmFAoFQdKE91msK63/tb/66qvw8PDAmjVr8OeffwIAOnTogD/++ANPP/20zgNkjLGGcvx6Lm7ll8HO0hSjQr3FDscghDR3QCtXG9zMK8W+uCw807m52CHp1B7VXnPBHiJH0vBaudrAwlSC0ko5buWXolUzW7FD0nv1Wlc7evRonDx5Evn5+cjPz8fJkyc5YWKMGRzVBPBxnX1gY2QjJg1Fua2KMsE0thJdTrEMZ2pKc8a6aq42UxMJ/D25RKcNrZOmqVOn4ujRow0QCmOMNZ6U/DIcScwBAEzkCeBaGVWTNP2TlIesIpnI0ejOvrgsEAGhPo5NZquYfyeDc9JUF1onTbm5uRgyZAh8fHwwf/58xMTENEBYjDHWsH45cxtEQO92zeDnaiN2OAbFx9kaT/g6gQjYEWM8o03G3tDyfngFnXa0Tpp27NiBzMxMvPPOOzh37hw6d+6MwMBAfPDBB7h161YDhMgYY7pVXinHH+dSAXCbgfoaHaacy7TNSEp0OVIZzt6qWTXXBOYzqagmg8dnSJvE9jiPq15zmpycnDB9+nQcPXoUt2/fxksvvYSff/4Zbdq00ep2jh8/jpEjR8LLywuCIGD79u3qc1VVVViwYAGCg4NhY2MDLy8vTJo0CRkZmptF+vr6QhAEja9Vq1ZpXBMbG4uePXvC0tISPj4+WL169T2xbN68Gf7+/rC0tERwcDD27t2r1WNhjBmOnZfSUVReBR9nK/Rp7yZ2OAZpeLAnzE0kSMgqxtVMwy/t7LvS9EpzAODvYQcTiYD80kpkSY2n1NpQHmuDpaqqKpw/fx5nzpzBrVu34O7urtXPl5aWIiQkBF9//fU958rKynDx4kW88847uHjxIrZu3YrExEQ89dRT91z77rvvIjMzU/01a9Ys9TmpVIpBgwahZcuWuHDhAj7++GMsX74cGzZsUF9z6tQpPP/885gyZQqio6MxatQojBo1CnFxcVo9HsaY/iMi/HRKOQF8YreWMOFl1vXiYG2Gfv7KhNMYRpt215TmRhh5Q8u7WZqZoE3NqrkrTWBPwcdVr+UiR44cwW+//Ya//voLCoUCY8aMwe7du9GvXz+tbmfo0KEYOnTofc85ODggMjJS49jatWsRHh6OlJQUtGjRQn3czs4OHh73H0799ddfUVlZiR9++AHm5uYIDAxETEwMPv30U0yfPh0A8MUXX2DIkCGYP38+AOC9995DZGQk1q5di/Xr12v1mBhj+u3C7TuIz5TCwlSCZ7twQ97HMbqTN/ZdycKOmHQsGOJvsAlojlSGc+rSXNNKmgDlvKbE7GJcyZBiQIB2gx9NjdYjTd7e3hg2bBjy8vKwYcMGZGdn44cffkD//v0bvDFcUVERBEGAo6OjxvFVq1bBxcUFYWFh+Pjjj1FdXa0+FxUVhV69esHc3Fx9bPDgwUhMTMSdO3fU1wwYMEDjNgcPHoyoqKgHxlJRUQGpVKrxxRjTf6o2A6NCveFobf6Iq9nD9GnfDA5WZsiWViAqKV/scOrt75pVc2EtHOHtaCV2OI0ugCeD15nWI03Lly/HuHHj7klcGppMJsOCBQvw/PPPw97eXn189uzZ6NSpE5ydnXHq1CksWrQImZmZ+PTTTwEAWVlZ8PPz07gtVRkxKysLTk5OyMrKuqe06O7ujqysrAfG8+GHH2LFihW6eniMsUaQI5Vhb03zQm4z8PgsTE0woqMnfj2Tgq3RaejR1lXskOqlKa6aq001GZzbDjya1iNN06ZNUydMqampSE1N1XVM96iqqsKzzz4LIsK6des0zr3xxhvo06cPOnbsiBkzZmDNmjX46quvUFFR0aAxLVq0CEVFReqvxngeGGOP539nU1GtIHRu6YQgbwexwzEKYzopezbtj8tCWWX1I67WP9lSGc7dbjoNLe9HNdKUXliOwrJKkaPRb1onTdXV1XjnnXfg4OAAX19f+Pr6wsHBAW+//Taqqqp0HqAqYbp9+zYiIyM1Rpnup2vXrqiurla3P/Dw8EB2drbGNarvVfOgHnTNg+ZJAYCFhQXs7e01vhhj+qtKrsCvZ5SlOW4zoDudWjihhbM1SivliIzPfvQP6Jm/L2eCCOjUwhFeTbA0BwAOVmbwcVY+9ngebXoorZOmWbNmYcOGDVi9ejWio6MRHR2N1atX4/vvv8fs2bN1GpwqYbp+/ToOHjwIFxeXR/5MTEwMJBIJ3NyUqzoiIiJw/PhxjYQuMjIS7du3h5OTk/qaQ4cOadxOZGQkIiIidPhoGGNi2n8lCznFFXC1tTD63esbkyAI6g7hWy8a3iq6vZeV0zCa6iiTSqAnl+jqQus5Tb/99ht+//13jVVvHTt2hI+PD55//vl7ymcPU1JSghs3bqi/T05ORkxMDJydneHp6YlnnnkGFy9exO7duyGXy9VzjJydnWFubo6oqCicOXMGffv2hZ2dHaKiojB37lxMmDBBnRC98MILWLFiBaZMmYIFCxYgLi4OX3zxBT777DP1/c6ZMwe9e/fGmjVrMHz4cPz+++84f/68RlsCxphh21TTZuCFri1gbvpY3VbYXUaHeePLQ9dx4noucosr0MzOQuyQ6oRLc/8K9LLHvitZPBn8EbR+5bCwsICvr+89x/38/DRWqNXF+fPnERYWhrCwMADK+UlhYWFYunQp0tPTsXPnTqSlpSE0NBSenp7qr1OnTqlj+f3339G7d28EBgZi5cqVmDt3rkay4+DggAMHDiA5ORmdO3fGvHnzsHTpUnW7AQDo3r07fvvtN2zYsAEhISHYsmULtm/fjqCgIG2fHsaYHrqaKcXZWwUwlQh4sWuLR/8A04qfqw1CfRyhIGDnpYxH/4CeUJXmOrd0arKlOZVAb96Dri60HmmaOXMm3nvvPfz444+wsFB+mqioqMDKlSsxc+ZMrW6rT58+D23b/qiW7p06dcLp06cfeT8dO3bEiRMnHnrNuHHjMG7cuEfeFmPM8KjaDAwO9IC7vaXI0RinMZ28EZNaiG3RaZjSw+/RP6AH9tSspGzqo0zAvyvoknJLUF4ph5W5icgR6Setk6bo6GgcOnQIzZs3R0hICADg0qVLqKysRP/+/TFmzBj1tVu3btVdpIwxVg9FZVXYXtOxmieAN5wRHb3w7q54xKVLcT27GG3d7cQO6aGyimQ4d0vZq29YE9pr7kHc7CzgamuOvJJKJGRJEdbCSeyQ9JLWSZOjoyPGjh2rcczHh7vqMsb00+YLqSivksPfww7hfs5ih2O0nG3M0ad9Mxy8moNt0el4a4i/2CE91N9xylGmzi2d4OnQtEtzgHJCf4CXA45fy0VcBidND6J10vTjjz82RByMMaZzCgXh59OqNgO+Db5rQVM3Oqw5Dl7NwY6YDLw5qD0kerytSlNvaHk/QV72OH4tF/E8GfyBeAkJY8xoHbuei9v5ZbCzNMWoMC+xwzF6/Tu4wc7CFOmF5Thbs5ebPsoqkuH8bWVpbiiX5tS4M/ijaT3S5Ofn99BPazdv3nysgBhjTFd+rpkA/mwXH1ib12t/cqYFSzMTDAv2xB/nU7HtYjq6tXp0bz0xqLbS6cKlOQ2BNZ3BE7KKUSVXwMyEx1XupvWryOuvv67xfVVVFaKjo7Fv3z7Mnz9fV3ExxthjuZ1fiiOJOQCACd14AnhjGd3JG3+cT8Xey5lY8XQgLM30bxWWatXc8I5cmquthbM1bC1MUVJRjaTcEvh78E4Xd9M6aZozZ859j3/99dc4f/78YwfEGGO68Mvp2yACerdrBj9XG7HDaTLCfZ3h7WiF9MJyHLqao3eJSWZROS6oSnPcGV6DRCIgwNMeZ28V4Eq6lJOm+9DZ2NvQoUPx119/6ermGGOs3sor5fjjnHIT7cndeZSpMUkkAp4OVc4f2xadJnI091Jtm/KErxM8HLhn191Um/fyvKb701nStGXLFjg783Jexpj4dsSkQyqrRgtna/Ru5yZ2OE3O6Jq96I4m5iK/pELkaDTt5YaWDxWoTpp4Bd39aF2eCwsL05gITkTIyspCbm4uvvnmG50Gxxhj2iIi/FQzAXxit5Yw0eNl78aqrbsdgrztEZcuxZ7LmZgU4St2SACAjEJlaU4QuDT3IKoVdPGZUhARt+m4i9ZJ06hRozS+l0gkaNasGfr06QN/f/1uZsYYM34Xbt/B1UwpLM0kGNeludjhNFmjw5ojLj0eWy+m603S9HdcTWmupTOX5h6grbstzE0kKJZVI7WgHC1crMUOSa9onTQtW7asIeJgjDGdUI0yPR3iDUdr7TYRZ7rzVIgXPth7FTGphbiZW4JWzWzFDgl7YpWbCfO2KQ9mZiJBOw9bxKVLcSWjiJOmu3ATBsaY0ciRyvB3zZyVibzPnKia2VmgRxtXAMD2mAyRo1GW5i6mFCpLczyf6aECPbnJ5YNw0sQYMxq/nU1BtYLQpaUTgrwdxA6nyRvTSTkhfHt0OohI1FhUE8CfaOkMd3suzT1MoDdPBn8QTpoYY0ahslqBX8+kAAAmdfcVNxgGABgU4AEbcxOkFJSpeyOJhRta1p1qMngcjzTdo05JU2xsLBQKRUPHwhhj9bb/ShZyiyvQzM4CQwJ5zoo+sDI3weAg5e9iW3S6aHGkF5YjWlWaC+J/G4/SwdMOggDkFlcgp1gmdjh6pU5JU1hYGPLy8gAArVq1Qn5+foMGxRhj2lLtM/dCeAuYm/Igur4YE6Zcwbg7NhMV1XJRYlDNc3vC1xluXJp7JGtzU7Sq6aLP85o01emVxdHREcnJyQCAW7du8agTY0yvxGdIcfZWAUwlAl7o2kLscFgtEa1d4G5vgaLyKhxJyBUlBlVpbgSX5upM3a+JkyYNdWo5MHbsWPTu3Ruenp4QBAFdunSBicn9N2G8efOmTgNkjLFH+fn0LQDA4CAPnuSrZ0wkAp4O9caG4zexPTodQxq5PJZ2p0xdmmvs+zZkgV722HkpgyeD36VOSdOGDRswZswY3LhxA7Nnz8a0adNgZ2fX0LExxtgjFZVVqefLTNaTJopM0+gwZdJ0OCEHRWVVcLA2a7T7/rtmr7lwX2e42XFCXVeqkSYuz2mqc3PLIUOGAAAuXLiAOXPmcNLEGNMLmy+kQlalgL+HHZ7wdRI7HHYfHTzt4e9hh4SsYuy5nNmoJVReNVc/qj3obueXQSqrgr1l4yW6+kzr2ZI//vijOmFKS0tDWpr+7WLNGGsaFArCz6eVE8And/flfbL0mGoT323RjfeekXanDDGpXJqrDycbc3jVbDVzlUeb1LROmhQKBd599104ODigZcuWaNmyJRwdHfHee+/xBHHGWKM6dj0Xt/PLYG9piqdDvcQOhz3E06HeEATg3K07SC0oa5T75NLc4wngEt09tE6alixZgrVr12LVqlWIjo5GdHQ0PvjgA3z11Vd45513GiJGxhi7r02nbgEAxnXxgbW51ltpskbk4WCJJ1vXbKvSSD2bdvOquceiKtFx0vQvrV9lfvrpJ3z33Xd46qmn1Mc6duwIb29v/Oc//8HKlSt1GiBjjN3PrbxSHL2mXMI+sRvvM2cIRoV54+SNPGyLTsfMfm0atJyaWlCGS6mFkAhQN9hk2vk3aeIVdCpajzQVFBTA39//nuP+/v4oKCjQSVCMMfYov5y+DSKgT/tm8K1pxMf025AgD1iaSXAzrxSX0hr2jfjvOOUoU7gfl+bqK7Bm/8YbOSWQVYnTmFTfaJ00hYSEYO3atfccX7t2LUJCQnQSFGOMPUx5pRx/nk8FwG0GDImthSkG12xx09Aluj2xqlVzPNetvrwcLOFobYZqBeFadrHY4egFrctzq1evxvDhw3Hw4EFEREQAAKKiopCamoq9e/fqPEDGGLvbjph0SGXVaOFsjd7tmokdDtPCqDBv7IjJwK5LGVgyvAPMTHS/5U1qQRkupRVBIoD3IXwMgiAgyMsBJ2/k4UqGFB2bO4odkui0/tfau3dvXLt2DaNHj0ZhYSEKCwsxZswYJCYmomfPng0RI2OMqRERfqrZZ25it5aQSLjNgCHp2cYVrrbmyC+txPFrDbOtyt6aCeBd/VzQzM6iQe6jqeB5TZrqtdzEy8uLJ3wzxkRx/vYdXM2UwtJMgnFdmosdDtOSqYkEI0O88OM/t7AtOh39O7jr/D72ckNLnQngFXQaeCtwxphB+ammzcCoUG84WpuLGwyrlzFhymQ3Mj4bUlmVTm9bozTHq+Yem2o7lYTMYsgVJHI04uOkiTFmMLKlMuyLUzYsnBjBbQYMVZC3Pdq42aKiWoF9NQ0odUW1bUq3Vi5wteXS3OPyc7WBlZkJyqvkSM4rETsc0XHSxBgzGL+dSUG1gvCEr5P6EzAzPIIg1NpWRber6FSluWHBXJrTBROJgA6eyq3TuESnZdJEREhJSYFMJmuoeBhj7L4qqxX47WwKAGAStxkweKptb04n5yOjsFwnt5mSX4ZYLs3pXCBvp6KmddLUpk0bpKamNlQ8jDGmQa4gRCXlY+WeeOQWV8DV1lzd64cZruZO1ujq5wwiYHuMbkab9sZxaa4h8Aq6f2mVNEkkErRt2xb5+fkNFQ9jjKnti8tEj48O4/n/nla3GZBVKXA4IVvkyJguqEt0F9NB9PiTjP9taMmlOV2qPdKki9+TIdN6TtOqVaswf/58xMXFNUQ8jDEGQJkwvfrLRWQWaU4HKKmoxqu/XMS+mlEFZriGBnvC3FSC6zklj136Sckvw+V0bmjZENp52MJUIqCwrAoZRU17eo7WSdOkSZNw9uxZhISEwMrKCs7OzhpfjDH2uOQKwopd8XjYZ9oVu+J5CbSBc7Ayw8CaPk2POyFctWouorULXLg0p1MWpiZo42YLALiS3rRLdFo3t/z8888bIAzGGPvX2eSCe0aYaiMAmUUynE0uQERrl8YLjOncqDBv7LmciZ2XMrBoqD9M67mtyp7LGQCA4cG811xDCPRyQEJWMa5kSDGoCY/kaZ00TZ48uSHiYIwxtZziupUA6nod01+92zWDk7UZcosr8E9Sfr32ErydX4q4dClMJAIGB+q+wzhTTgb/6yJPBq9XSp+UlIS3334bzz//PHJycgAAf//9N65cuaLT4BhjTZObnaVOr2P6y9xUua0KAGy7mFav21CX5lpxaa6hBHlz2wGgHknTsWPHEBwcjDNnzmDr1q0oKVF2CL106RKWLVum8wAZY01PuJ8z3B6y0aoAwNPBEuF+PI/SGIyqWUW3/0o2Siuqtf55bmjZ8FQNLjOLZCgorRQ5GvFonTQtXLgQ77//PiIjI2Fu/u++T/369cPp06d1GhxjrGkykQho6WJ933NCzX+XjQyAiUS47zXMsIT5OMLP1QblVXLsv6Ldtiq38rg01xjsLM3gW/M32ZRLdFonTZcvX8bo0aPvOe7m5oa8vDydBMUYa9qOJubg3K07EAC42GpuyuvhYIl1EzphSBCPKhgLQRAwKrR+26qoSnPdedVcg+PO4PWYCO7o6IjMzEz4+flpHI+Ojoa3t7fOAmMMUC49P5tcgJxiGdzslOUYHl0wbmWV1Xh7u7IP3Ms9/LB4WAf+N9AEjArzwmcHr+GfG3nIlsrgbl+3+Wpcmms8AV722HM5k5MmbYwfPx4LFizA5s2bIQgCFAoF/vnnH7z55puYNGlSQ8TImqh9cZlYsSteY+m5p4Mllo0M4FEGI/b5wetIu1MOb0crvDGwHUwkArcVaAJautigc0snXLh9BztjMjCtV6tH/sytvFJcyVCV5pruMvjGwtup1KM898EHH8Df3x8+Pj4oKSlBQEAAevXqhe7du+Ptt99uiBhZE/SgbtBZRTLuBm3E4tKL8N2JmwCA90cFwcZC6891zICptlXZWscSXe3SnLON+SOuZo9LVZ5Lziut14R9Y6B10mRubo7//ve/SEpKwu7du/HLL78gISEBP//8M0xMTBoiRtbEPKwbtOoYd4M2PtVyBRZtvQwFASNDvNDX303skFgjGx7sCTMTAVczpUjIenQJSL3XHJfmGkUzOwu42VmACHX6/Rij+rVeBdCiRQsMHToU48aNQ9u2bXUZE2vitOkGzYzHxlO3cDm9CPaWplg6IkDscJgInGzM0be9Mll+1ITw5LxSxGcqS3NNuUN1Y/u3RMdJU519//33CAoKgqWlJSwtLREUFITvvvtO17GxJoq7QTc9qQVlWHPgGgBgyfAOaPaQHk3MuI3ppCzR7YjOeOho8l4uzYlCvYIuvWkmTVpPGFi6dCk+/fRTzJo1CxEREQCAqKgozJ07FykpKXj33Xd1HiRrWrgbdNNCRHhnRxzKq+To6ueMZ7v4iB0SE1FffzfYW5oiSyrDmZv56N7G9b7X7a4pzY3oyKW5xqQeacpsmpPBtU6a1q1bh//+9794/vnn1ceeeuopdOzYEbNmzeKkiT22cD9neDpYIqtIdt95TQKUvXq4G7Rx2BWbiaOJuTA3keCDMcEQBG4n0JRZmJpgeEcv/O9sCrZGp983abqZW4KrqtJcAJfmGpNqpOlaVgkqqxUwN633LB+DpPWjraqqQpcuXe453rlzZ1RXN83Z9Ey3TCQClo0MuG/CBCjnNC0dwd2gjUFhWSXe3aXcs3JmvzZo3cxW5IiYPlCtovv7cibKK+X3nFeV5p5s4wonLs01Kh9nK9hZmqJSrsD1nGKxw2l0WidNEydOxLp16+45vmHDBrz44os6CYqxJ3ydYfGQTzBl93khZYbnw70JyCupRFs3W8zo3VrscJie6NLSCc2drFBaKUfk1ex7zu+5rNxqZXgwjzI1NkEQmvRk8DqV59544w31/wuCgO+++w4HDhxAt27dAABnzpxBSkoKN7dkOrPuaBIqqhUI8rLHkuEdkFNcATc7S5y7lY9PI69j6Y44dPF1QksXG7FDZfUUlZSPP86nAgA+HBPc5Ib52YNJJAJGh3njq8M3sO1iGp4K8VKfU5XmTLk0J5pALwecvlmAeE6a7i86Olrj+86dOwMAkpKSAACurq5wdXXFlStXdBwea4oyi8qx6fRtAMD8If6IaP3vnIZwP2ecuJ6Hc7fuYO4fMfjzlQiYmvCbraGRVcmxZNtlAMCEbi3QxZfnpzFNo2qSpuPX85BbXKFeUcmlOfE15c7gdUqajhw50tBxMKb21eEbqKxWINzXGb3aak4CNZEI+Oy5UAz9/AQuphRi7ZEbeH1AO5EiZfX1zZEbuJlXCjc7C7w1xF/scJgeat3MFiHNHXAprQi7YzPwf08q9zvdzQ0tRaeaDB6fIYVCQZA0ofml/BGd6ZWU/DL8eU5ZsnlzcPv7rqRq7mSN90cHAQC+PHQdF27fadQY2eO5ll2MdceUo9TvPh0Ie0szkSNi+ko1IVzV6DIptwQJWcXK0lygu5ihNWmtm9nAwlSC0ko5bheUiR1Oo9I6aZLJZPj4448xbNgwdOnSBZ06ddL4YuxxfH7wGqoVhF7tmj20pcDTod54OtQLCgJe/yMaxbKqRoyS1ZdCQVi09TKq5ISBAe68ySp7qBEhXjCRCIhNK8KNnBLsjf23NOdozaU5sZiaSODvYQeg6ZXotO7TNGXKFBw4cADPPPMMwsPDuacK05nr2cXYFqP8RPnmoEeX3N59Ogjnb91BakE5lu+Mx5pnQxo6RPaYfj2bggu378DG3ATvPh3Irx/soVxtLdC7XTMcTsjB2sPX1VsnDeVVc6IL8FKWTq9kSDGio9ejf8BIaD3StHv3bmzfvh3r1q3D8uXLsWzZMo0vbRw/fhwjR46El5cXBEHA9u3b1eeqqqqwYMECBAcHw8bGBl5eXpg0aRIyMjI0bqOgoAAvvvgi7O3t4ejoiClTpqCkpETjmtjYWPTs2ROWlpbw8fHB6tWr74ll8+bN8Pf3h6WlJYKDg7F3716tHgt7fJ9GXgMRMCTQAx2bOz7yegcrM3z2XCgkAvDXxTTsjs145M8w8WQVybD67wQAwFtD/OHpYCVyRMwQ+LkqV8huj8lARs2elJ9FXsO+uEwxw2rymmrbAa2TJm9vb9jZ2enkzktLSxESEoKvv/76nnNlZWW4ePEi3nnnHVy8eBFbt25FYmIinnrqKY3rXnzxRVy5cgWRkZHYvXs3jh8/junTp6vPS6VSDBo0CC1btsSFCxfw8ccfY/ny5diwYYP6mlOnTuH555/HlClTEB0djVGjRmHUqFGIi4vTyeNkj3Y5rQh/x2VBEIA36jDKpBLu54z/9GkDAFi89TIyCssbKkT2mJbvvILiimqE+jhiQreWYofDDMC+uEz8cDL5nuM50gq8+stFTpxEpEqa4jOKQPTgPQKNjUBaPtq///4bX375JdavX4+WLXX3wicIArZt24ZRo0Y98Jpz584hPDwct2/fRosWLXD16lUEBATg3Llz6i7l+/btw7Bhw5CWlgYvLy+sW7cOS5YsQVZWFszNlTXwhQsXYvv27UhIUH7qfe6551BaWordu3er76tbt24IDQ3F+vXr7xtLRUUFKioq1N9LpVL4+PigqKgI9vb2j/t0NDmTfziLY9dyMTrMG589F6rVz1bJFXhm3SlcSitCt1bO+HVqN+4Wrmf2X8nCKz9fgKlEwK5ZPdDBk/9G2MPJFYQeHx1GZtH9N+ZWbad0ckE//nsXQXmlHIHL9kFBwJnF/eFub7h7gUqlUjg4ONTp/VvrkaYuXbpAJpOhVatWsLOzg7Ozs8ZXQyoqKoIgCHB0dASg3CjY0dFRY1uXAQMGQCKR4MyZM+prevXqpU6YAGDw4MFITEzEnTt31NcMGDBA474GDx6MqKioB8by4YcfwsHBQf3l48ObjNbXuVsFOHYtF6YSAa8PaKv1z5uZSPD5+DBYmZng9M0C/PfEzQaIktVXsawKy3Yoe7hN79WKEyZWJ2eTCx6YMAHK7ZQyi2TqeU6scVmZm6i3PWpKk8G1ngj+/PPPIz09HR988AHc3d0bbSKnTCbDggUL8Pzzz6szwaysLLi5uWlcZ2pqCmdnZ2RlZamv8fPz07jG3d1dfc7JyQlZWVnqY7WvUd3G/SxatEijU7pqpIlph4jw8f5EAMCzT/jUu8O3n6sNlj8VgAV/XcaaA4no0cYVQd4OugyV1dMn+xORJZXB18Uas/trnxSzpimn+MEJU32uY7oX6GWP6zkliEuXop9/02gBoXXSdOrUKURFRSEkpPFWKlVVVeHZZ58FEd133zsxWFhYwMLCQuwwDN6J63k4m1wAc1MJZvVr81i39WwXHxxOyMH+K9mY/Xs09szqCStzEx1FyurjYsoddXf3D0YHw9KMfx+sbtzs6lbuqet1TPeCvB2wPSajSY00aV2e8/f3R3l54022VSVMt2/fRmRkpEa90cPDAzk5ORrXV1dXo6CgAB4eHuprsrM1N3xUff+oa1TnWcMgInxyQDnKNLFby8deTSUIAlaN6Qh3ewvczC3F+3vidREmq6fKagUW/XUZRMAznZujexvXR/8QYzXC/Zzh6WCJB9UyBACeDpYP7efGGlZAE1xBp3XStGrVKsybNw9Hjx5Ffn4+pFKpxpcuqRKm69ev4+DBg3BxcdE4HxERgcLCQly4cEF97PDhw1AoFOjatav6muPHj6Oq6t/mh5GRkWjfvj2cnJzU1xw6dEjjtiMjIxEREaHTx8M07b+Sjdi0Ilibm+DVPrrZ4d7JxhyfjFOOgv56JgUH4+/dIZ01jv+euInE7GI425hjybAOYofDDIyJRMCykQEAcE/ipPp+2cgAngQuokBP5RSItDvlKCprGg2GtU6ahgwZgqioKPTv3x9ubm5wcnKCk5MTHB0d1UlIXZWUlCAmJgYxMTEAgOTkZMTExCAlJQVVVVV45plncP78efz666+Qy+XIyspCVlYWKisrAQAdOnTAkCFDMG3aNJw9exb//PMPZs6cifHjx8PLS9ls64UXXoC5uTmmTJmCK1eu4I8//sAXX3yhMR9pzpw52LdvH9asWYOEhAQsX74c58+fx8yZM7V9elgdyRWETyOVo0wvP+kHV1vdlTp7tm2GqT2U89je+iuW5zyIIDmvFF8cug4AWDoigDdWZfUyJMgT6yZ0goeDZgnOw8ES6yZ0wpAg3n9OTA7WZmjupKwQXMlsGiU6rVsOHDt27KHne/fuXefbOnr0KPr27XvP8cmTJ2P58uX3TOBWOXLkCPr06QNA2dxy5syZ2LVrFyQSCcaOHYsvv/wStra26utjY2Px2muv4dy5c3B1dcWsWbOwYMECjdvcvHkz3n77bdy6dQtt27bF6tWrMWzYsDo/Fm2WLDJge3Q6Xv8jBvaWpjixoB8crHS7/1hFtRxPr/0HCVnF6N2uGTb+3xPcfbqREBFe+O8ZRN3MR8+2rtj0Mu8cwB6PXEE4m1yAnGIZ3OyUJTkeYdIPr/x8HvuvZOPt4R0wtWcrscOpF23ev7VOmtj9cdJUd1VyBfqvOYaUgjLMH9wer/V9vAngD3ItuxgjvzqJimoFlo8MwEtP3j8JZ7q1+Xwq5m+JhaWZBAde740WLtZih8QYayBfHrqOTyOv1avHnr7Q5v1b69Vzx48ff+j5Xr16aXuTrInZfD4NKQVlcLU1x/896dtg99PO3Q6Lh3XAsp1X8MHfCejexhXt3HXTzZ7dX15JBVbuvQoAmDugHSdMjBm5f7dTaRrlOa2TJlVZrLbaQ+9yufyxAmLGTVYlx1eHlXNd/tOnDazNtf4nqJVJES1xJDEHRxNzMft/0dj+2pO87L0Bvb87HoVlVQjwtMeUHjyyx5ixC/RSTgZPyi2FrEpu9K+vWk8Ev3PnjsZXTk4O9u3bhyeeeAIHDhxoiBiZEfn1TAoyi2TwdLDEC11bNPj9CYKA1c90hIuNORKyitWNNJnuHbuWi+0xGZAIwKqxwTA10frlhTFmYNztLeBiYw65gpCQVSx2OA1O61e12luHODg4wNXVFQMHDsRHH32Et956qyFiZEaitKIa3xy5AQCY079to30icbOzxOpnOgIAvj+ZjBPXcxvlfpuSsspqLNl2GQDwf0/6oWNzR3EDYow1CkEQavVrMv4Snc4+Crq7uyMxkT/Fswf78Z9k5JdWwtfFGmM7N2/U++7fwR0TuilHtub9eQkFpZWNev/G7vOD15F2pxzejlZ4Y2A7scNhjDUiVYmuKTS51HpCSWxsrMb3RITMzEysWrUKoaGhuoqLGZmisip8e1y5ke7cge1gJkLpZsmwAEQl5SMptxSLtsZi/YTOvBReB+LSi/BdzSbJ748Kgo1Fw85TY4zpl8Am1Blc61e30NBQCIKAuzsVdOvWDT/88IPOAmPGZcOJJBTLqtHe3Q4jO3qJEoOVuQm+GB+G0d/8g/1XsvHHuVSMD2/4eVXGrFquwKKtl6EgYERHT/T1d3v0DzHGjIpqc/SETCmq5Qqjns+oddKUnJys8b1EIkGzZs1gacmbJrL7yyupwI//3AIAzBvUDhIRm9IFeTtg3qD2WPV3Albsike4nzNaNbN99A+y+9p46hYupxfB3tIUS2u2vGCMNS0tna1ha2GKkopqJOWWor2H8bZ20TodbNmypcaXj48PJ0zsob45koSySjlCfBwxMMBd7HAwvWcrRLRyQXmVHHP/iEGVXCF2SAYptaAMaw5cAwAsHtaBd5tnrImSSAR08FQmSsY+Gbxekw8OHTqEQ4cOIScnBwqF5hsOl+hYbRmF5fjl9G0AwJuD2unFHCKJRMCaZ0Mw9IsTuJRWhC8OXsebg9uLHZZBISK8syMO5VVyhPs549kuPmKHxBgTUaCXA87duoMrGVKM6SR2NA1H65GmFStWYNCgQTh06BDy8vLu6dvEWG1fHb6OSrkCXf2c0aONq9jhqHk5WuGD0cEAgK+P3sCZm/kiR2RYdsdm4mhiLsxNJPhgdLCoJVfGmPiaStsBrUea1q9fj40bN2LixIkNEQ8zIrfySvHn+TQAwPzB7fVilKm24R09cSSxObZcSMMbf17C3jk9db5xsDEqLKvEil1XAAAz+7VBGzeeE8ZYU6daQRefIQUR6d3rva5oPdJUWVmJ7t27N0QszMh8fvAa5ApC3/bN0MXXWexw7mv5U4Fo4WyN9MJyLN0RJ3Y4BuHDvQnIK6lEWzdbzOjdWuxwGGN6oK2bHcxMBEhl1Ui7Uy52OA1G66Rp6tSp+O233xoiFmZEErOKseNSBgBg3iD9nS9ka2GKz54LhYlEwI6YDGyPThc7JL0WlZSPP86nAgA+HBMMc1PjXVrMGKs7c1OJekN0Yy7RaV2ek8lk2LBhAw4ePIiOHTvCzEyznPHpp5/qLDhmuNYcSAQRMCzYQ93DQ191bumEWf3a4POD1/HO9jh0bukEH2drscPSO7IquXqrlBe7ttDb0UPGmDgCvexxJUOKKxlSDAnyFDucBlGvjuCqzt9xcZrlDGOtYTLtXEotxIH4bEgEGMyWGjP7tsHxa7m4mFKIN/6Mwe/TI2DCk5s1fHPkBm7mlcLNzgJvDfEXOxzGmJ5RbqeSZtSdwbVOmo4cOdIQcTAj8skB5R6Eo8K80cbNMJqcmZpI8PlzYRj25Qmcu3UH647ewMx+bcUOS29cyy7GumNJAIAVTwXyhHnG2D0Cm8AKOp6QwHTq9M18nLieBzMTAXMHGMYok0oLF2useCoQAPDZweuISS0UNyA9oVAQFm29jCo5YUAHdwwJ8hA7JMaYHurgaQ9BALKlFcgtrhA7nAbBSRPTGSLCJ/uVo0zPPeFjkPOCxnTyxoiOnpArCK//Ho3SimqxQxLdb2dTcOH2HdiYm+DdpwO5DM8Yuy8bC1P4udgAMN7RJk6amM4cvZaL87fvwMJUglkGWtoSBAErRwXDy8ESt/LL8O6ueLFDElW2VIaP/k4AoOy15eVoJXJEjDF9Fliz8MdY5zVx0sR0goiwpmYu06SIlnC3N9x9yByszbDm2VAIAvDH+VTsi8sUOyTRLNtxBcUV1Qj1ccTECF+xw2GM6bnaTS6NESdNTCf2xWUhLl0KG3MTvNqnjdjhPLaI1i54pZeycePCrZeRVSQTOaLGt/9KFvZdyYKpRMCHY4J5NSFj7JGMfTJ4nVbP7dy5s843+NRTT9U7GGaY5ArCmkjlbvdTeraCs425yBHpxhsD2+HkjVzEpUsxb3MMfn65a5PZY61YVoVlO5RbpUzv1QodPO1FjogxZgiUbQeAW/llKJZVwc7SuFba1ilpGjVqVJ1uTBAEyOXyx4mHGaDt0em4kVMCByszTO3pJ3Y4OmNuKsEX48Mw/MsT+OdGPn74JxlTe7YSO6xG8cn+RGRJZWjpYo3Z/Q1zfhpjrPE525jD08ESmUUyXM0sRrifcTXBrVN5TqFQ1OmLE6amp7Jagc8PKUeZZvRuDXsj+1TRupkt3hkRAABYvS/RaOv0tV1MuYNNp28DAD4YHQxLMxORI2KMGRJjLtHxnCb2WP48n4rUgnK42lpgcveWYofTIF4Ib4EBHdxRKVdgzu/RkFUZ74eDKrkCi/66DCJgbKfmeLKNq9ghMcYMTICX8a6g07ojOACUlpbi2LFjSElJQWVlpca52bNn6yQwpv9kVXJ8dfg6AGBWvzawNq/XPye9JwgCPhobjCFfFOJ6Tgk+3HsVK54OEjusBrHh+E0kZhfD2cYcS4Z3EDscxpgB+nekiZMmREdHY9iwYSgrK0NpaSmcnZ2Rl5cHa2truLm5cdLUhPwcdRvZ0gp4O1phfLiP2OE0KBdbC3z8TEe89OM5/BR1G33au6Gvv5vYYelUcl4pvjikTILfGdHBaCb0M8Yalyppup5djIpqOSxMjafEr3V5bu7cuRg5ciTu3LkDKysrnD59Grdv30bnzp3xySefNESMTA+VVFSr9yKb07+tUf1RPEif9m54qbsvAGD+lkvIKzGebQKICEu2XUZltQI927piVKi32CExxgyUt6MVHKzMUK0gXM8uETscndI6aYqJicG8efMgkUhgYmKCiooK+Pj4YPXq1Vi8eHFDxMj00A8nk1FQWolWrjYY06npvMEuHOqP9u52yCupxFtbYkFEYoekE1supOFUUj4szSRYOSqYt0phjNWbIAhGOxlc66TJzMwMEonyx9zc3JCSkgIAcHBwQGpqqm6jY3qpsKwS/z1+EwAwd2A7mJo0nfUElmYm+Hx8KMxNJDickINfzqSIHdJjyyupwMq9VwEAcwe0QwsXw9szkDGmX4x1XpPW73ZhYWE4d+4cAKB3795YunQpfv31V7z++usICjLOybFM07fHb6K4ohodPO0xPNhT7HAaXQdPeywY6g8AeH93PG7kFIsc0eN5f3c8CsuqEOBpjyk9jKfPFmNMPKoml3HpTXyk6YMPPoCnp/KNcuXKlXBycsKrr76K3NxcfPvttzoPkOmXnGIZfvwnGQAwb2C7JtMh+27/190XPdu6oqJagdn/i0FFtWG2ITh2LRfbYzIgEYAPxwQ3qVFDxljDCfJWjjRdzSyGXGEc0xiAeqye69Kli/r/3dzcsG/fPp0GxPTbN0eSIKtSINTHEf07GNfqMW1IJALWjAvB4M+PIz5Tik8PXMOiYYa1RL+sshpLtl0GALzU3Q8hPo7iBsQYMxp+rrawMjNBeZUcyXmlaONmK3ZIOqH1x8p+/fqhsLDwnuNSqRT9+vXTRUxMT6UXluO3mjk88we3b/KThd3sLbFqbEcAwIYTN3HqRp7IEWnni4PXkXanHN6OVpg3qJ3Y4TDGjIiJRIC/px0A45oMrnXSdPTo0XsaWgKATCbDiRMndBIU009fHryOSrkC3Vu7cKfoGoMDPfB8uA+IgDf+vITCsnv/NvRRXHoRvjupLLO+NyoQNhbG2ZiUMSYe1WRwY9p+qs6vlLGxser/j4+PR1ZWlvp7uVyOffv2wdu76Sw9b2pu5pZgy8U0AMCbg9uLHI1+eWdEAM7cLMDNvFIs3nYZX7/QSa9H4arlCizaehlyBWF4R0/083cXOyTGmBEKNMLtVOqcNIWGhkIQBAiCcN8ynJWVFb766iudBsf0x2cHr0OuIPT3d0OnFk5ih6NXrM1N8fn4UIz55hT2Xs7ClgtpGNdFfzukbzx1C5fTi2BvaYplIwPEDocxZqRq92oiIr3+MFlXdU6akpOTQURo1aoVzp49i2bNmqnPmZubw83NDSYmxt8Vuim6minFrksZAIA3eO7LfXVs7oi5A9vh4/2JWL7zCsL9nNHSxUbssO6RWlCGNQeuAQAWD+sANztLkSNijBmrdu52MJEIuFNWhcwiGbwcrcQO6bHVOWlq2VK5g71CoWiwYJh+Ur3JDu/oqR5uZfea0bs1jl3LxdnkAsz5PQabZ0TATI+W8BMR3tkRh/IqOcL9nPGsHo+GMcYMn6WZCdq62SIhqxhXMqRGkTTV6xU9KSkJs2bNwoABAzBgwADMnj0bSUlJuo6N6YHolDs4eDUbEgF4YyCPMj2MiUTAZ8+Fws7SFDGphfjq8A2xQ9KwOzYTRxNzYW4iwQejg5tsjy3GWOMJMLLtVLROmvbv34+AgACcPXsWHTt2RMeOHXHmzBkEBgYiMjKyIWJkIvrkQCIAYGyn5mjdzDj6bDQkb0crrBwdDABYe/g6LtwuEDkipaKyKqzYdQUA8FrfNkbTM4Uxpt+MbTK41uuMFy5ciLlz52LVqlX3HF+wYAEGDhyos+CYuE4l5eGfG/kwMxEwu39bscMxGE+FeOFIQg62Radjzu8x+HtOT9hZmoka04d/X0VeSSXauNliRp9WosbCGGs6jK3tgNYjTVevXsWUKVPuOf7yyy8jPj5eJ0Ex8RERPtmvHGV6PrwFfJx5E1dtrHg6EM2drJB2pxzLdl4RNZbTN/Px+znlZtofjgmGhSkv2GCMNQ5VeS69sBx3Sg2jj93DaJ00NWvWDDExMfccj4mJgZtb091Ww9gcSczBxZRCWJpJMLNvG7HDMTj2lmb4/LlQSARg68V07KxZfdjYZFVyLN6q3Crlha4t8ISvsyhxMMaaJntLM7R0UX7oNoYSXZ2TpnfffRdlZWWYNm0apk+fjo8++ggnTpzAiRMnsGrVKrzyyiuYNm1aQ8bKGolCQfhkv3LF3OTuvnCz52Xp9dHF11mdcC7ZdhnpheWNHsM3R27gZl4p3OwssGCIf6PfP2OMBRrRZPA6J00rVqxASUkJ3nnnHSxduhRfffUVevfujd69e2Pt2rVYvnw53n777YaMlTWSvXGZiM+Uws7CFDN6tRY7HIM2q39bhPo4olhWjTf+iGnU3b6vZRdj3THlqtYVTwXCwUrceVWMsabJmCaD1zlpIlK+2AuCgLlz5yItLQ1FRUUoKipCWloa5syZYxTdPpu6arkCn0YqR5mm9PSDk425yBEZNjMTCT5/LhTW5iY4k1yAb483TmsOhYKwaOtlVMkJAzq4Y0iQR6PcL2OM3c2Y2g5oNafp7qTIzs4OdnZ2Og2IiWtbdDpu5pbCydoMU3r4iR2OUfB1tcHypwIBAJ8euIbLaQ3/wvHb2RRcuH0HNuYmePfpQP5AwxgTjao8dzOvFGWV1SJH83i0SpratWsHZ2fnh34xw1VRLcfnB68DAF7t01r0ZfLGZFzn5hga5IFqBWHO79EN+sKRLZXho78TAADzB7c3ii68jDHD5WZniWZ2FiACrmYWix3OY9GqT9OKFSvg4MDbaBirP86lIr2wHG52FpgU4St2OEZFEAR8OCYY0SmFuJlXivf3XMUHNU0wdW35zisorqhGiI8jJvLvkTGmBwK97HE0MRfxGUXo3NJwN33XKmkaP348txUwUuWVcvW2H7P6tYGlGffy0TVHa3OseTYEL353Br+dSUGfds0wKFC3c40OXMnC33FZMJUIWDUmGCa8VQpjTA+okiZDnwxe5/Icz4kwbpuibiG3uALNnazw3BMtxA7HaD3ZxhXTeyk7ci/cehk5UpnObrtYVoWlO5SNNKf1aoUOnvY6u23GGHscxrKCTuvVc8z4FMuq1EvT5/RvC3PTeu3jzOpo3qB2CPC0R0FpJeZtvgSFjtoQfLI/EVlSGVq6WGMOb3vDGNMjqsngiVnFqJIrRI6m/ur87qhQKLg0Z6S+P5mMwrIqtG5mg9Fh3mKHY/QsTE3wxfhQWJhKcOJ6HjaeuvXYt3kx5Q42nb4NAFg5KpjLq4wxveLjZA07C1NUyhW4kVMidjj1xkMKTdyd0kp8dyIZAPDGwPYwNeF/Eo2hrbsd3h7eAQCwal8CErLqP2RdJVdg0V+XQQSM7dQcPdq66ipMxhjTCYlEQAd1vybDLdHxO2QTt/5YEkoqqhHgaY+h3ACxUU3o1hL9/N1QWa3AnP/FQFYlr9ftbDh+E4nZxXC2MceSmkSMMcb0japEF5duuE0uOWlqwnKkMvwUdQsA8ObgdpDwSqtGJQgCPhrbEa625kjMLsbqfYla30ZyXim+OKTsrfXOiA5w5g7ujDE9FVQzGTyeR5qYIVp75AZkVQp0bumEvu15vpoYmtlZ4ONnQgAAP/yTjOPXcuv8s0SEJdsuo7JagZ5tXTEqlOejMcb0V6C3cqQpPlOqswUwjU3UpOn48eMYOXIkvLy8IAgCtm/frnF+69atGDRoEFxcXCAIAmJiYu65jT59+kAQBI2vGTNmaFyTkpKC4cOHw9raGm5ubpg/fz6qqzU7Mh89ehSdOnWChYUF2rRpg40bN+r40eqX1IIy/O9sCgDgzUHtuaWEiPr6u2FSREsAwLzNl1BQWlmnn/vrYjpOJeXD0kyClaOC+XfIGNNrrZvZwtxUgpKKaqQUlIkdTr2ImjSVlpYiJCQEX3/99QPP9+jRAx999NFDb2fatGnIzMxUf61evVp9Ti6XY/jw4aisrMSpU6fw008/YePGjVi6dKn6muTkZAwfPhx9+/ZFTEwMXn/9dUydOhX79+/XzQPVQ18cuo4qOaFHG1dEtHYRO5wmb/GwDmjjZovc4gos+Cv2kS0+8ksq8P6eeADA6wPaoYWLdWOEyRhj9WZmIoG/h3K/WkOdDK5VR3BdGzp0KIYOHfrA8xMnTgQA3Lp166G3Y21tDQ+P+09iPnDgAOLj43Hw4EG4u7sjNDQU7733HhYsWIDly5fD3Nwc69evh5+fH9asWQMA6NChA06ePInPPvsMgwcPrt+D02M3ckqw9WIaAODNwe1FjoYBgKWZsg3BqK//QWR8Nn4/l4rnwx/cZPS93fEoLKtCB0973liZMWYwAr3sEZtWhCsZRRje0VPscLRmFHOafv31V7i6uiIoKAiLFi1CWdm/w35RUVEIDg6Gu7u7+tjgwYMhlUpx5coV9TUDBgzQuM3BgwcjKirqgfdZUVEBqVSq8WUoPjt4DQoCBnRwR6iPo9jhsBqBXg54a7A/AODdXfFIyr1/L5Nj13KxPSYDEgFYNSYYZtwmgjFmIAIMvDO4wb/avvDCC/jll19w5MgRLFq0CD///DMmTJigPp+VlaWRMAFQf5+VlfXQa6RSKcrLy+97vx9++CEcHBzUXz4+Prp8WA3mSkYR9sRmQhCUnamZfpnSww9PtnFBeZUcr/8eg8pqzc65ZZXVWLLtMgDgpe5+COGklzFmQAINvFeTqOU5XZg+fbr6/4ODg+Hp6Yn+/fsjKSkJrVu3brD7XbRoEd544w3191Kp1CASpzUHrgEARnb04r3J9JBEImDNuFAM/vw4LqcX4fOD1zBvUHucTS5ATrEMB+OzkXanHN6OVpz0MsYMTgcPe0gEIK+kAjlSGdzsLcUOSSsGnzTdrWvXrgCAGzduoHXr1vDw8MDZs2c1rsnOzgYA9TwoDw8P9bHa19jb28PKyuq+92NhYQELCwtdh9+gLty+g8MJOTCRCJg7kN9w9ZWHgyVWjQnGq79exDdHk/D7udR7VtSNCvOCjYXR/fkyxoyclbkJWjWzxY2cElzJkBpc0mTw5bm7qdoSeHoqJ5hFRETg8uXLyMnJUV8TGRkJe3t7BAQEqK85dOiQxu1ERkYiIiKicYJuJJ/sVzZPfKZTc/i52ogcDXuYocGe6F6zqvF+LQi+OZKEfXGZjR0WY4w9tn9LdIbXGVzUpKmkpAQxMTHqRCc5ORkxMTFISVH2DyooKEBMTAzi45VLqxMTExETE6Oei5SUlIT33nsPFy5cwK1bt7Bz505MmjQJvXr1QseOHQEAgwYNQkBAACZOnIhLly5h//79ePvtt/Haa6+pR4pmzJiBmzdv4q233kJCQgK++eYb/Pnnn5g7d24jPyMN558beYi6mQ9zEwlmD2grdjjsEeQKeuBEcJUVu+IhN9AGcYyxpsuQ5zWJmjSdP38eYWFhCAsLAwC88cYbCAsLU/dQ2rlzJ8LCwjB8+HAAwPjx4xEWFob169cDAMzNzXHw4EEMGjQI/v7+mDdvHsaOHYtdu3ap78PExAS7d++GiYkJIiIiMGHCBEyaNAnvvvuu+ho/Pz/s2bMHkZGRCAkJwZo1a/Ddd98ZTbsBIsLHNaNML3RtAW/H+5ccmf44m1yAbGnFA88TgMwiGc4mFzReUIwxpgOBNSvo4gxwpEmgR3XRY3UilUrh4OCAoqIi2Nvr1wTryPhsTNt0HlZmJjj+Vl80szOsuVhN0Y6YdMz5PeaR130xPhRP8/YpjDEDUlhWidB3IwEAl5YNgoOVmajxaPP+bXRzmpgmhYKw5oBylOmlJ305YTIQbnZ1mxxZ1+sYY0xfOFqbqysehrZ5LydNRm735UwkZBXDzsIUr/RqJXY4rI7C/Zzh6WCJB+0mJwDwdLBEuJ9zY4bFGGM6YaiTwTlpMmLVcgU+i1T2ZZreqxUcrc1FjojVlYlEwLKRytWddydOqu+XjQyAiYQ36WWMGR7VvCYeaWJ646+LaUjOK4WzjTn+j/cnMzhDgjyxbkIneDholuA8HCyxbkInDAkyvH2bGGMMMNwVdNwdz0hVVMvx5aEbAID/9GkNW26EaJCGBHliYICHuiO4m52yJMcjTIwxQxborUyabuSWQFYlh6WZicgR1Q2/kxqp/51JQXphOdztLTChW0uxw2GPwUQiIKKm0SVjjBkDD3tLONuYo6C0EolZxQazjyaX54xQWWU11h5JAgDM6tfWYDJ4xhhjTYMgCAZZouOkyQj9dOo28koq0MLZGs920f9NhBljjDU9AQa4go6TJiNTVF6F9ceUo0yvD2gLc1P+FTPGGNM/qhV0PNLERPP9iZsoKq9CWzdb7hTNGGNMb6nKcwlZUoPZR5OTJiOSX1KB708mAwDeGNiOV1gxxhjTW34uNrA2N4GsSoGbj9igXF9w0mRE1h9LQmmlHEHe9hgS5CF2OIwxxtgDSSQCOngqR5sMZfNeTpqMRFaRDJuibgMA3hzUHoLAo0yMMcb0W5BqMni6Ycxr4qTJSHx1+DoqqhV4wtcJvds1Ezscxhhj7JEMbTI4J01GICW/DH+cSwXAo0yMMcYMR+22A0T6PxmckyYj8Pmha6hWEHq2dUXXVtw5mjHGmGFo524HMxMBUlk10u6Uix3OI3HSZOCuZxdje3Q6AGD+4PYiR8MYY4zVnbmpBG3d7AAYRomOkyYD92nkNSgIGBzojo7NHcUOhzHGGNOKql9TvAGsoOOkyYDFpRfh77gsCAIwbxCPMjHGGDM8hrQHHSdNBuyTA4kAgKdDvNDO3U7kaBhjjDHtBXobzgo6TpoM1PlbBTiamAsTiYDXB7QTOxzGGGOsXjp42kMQgCypDPklFWKH81CcNBkgIsLq/cpRpme7+MDX1UbkiBhjjLH6sbUwha+L8n1M30ebOGkyQCeu5+FscgHMTSWY3b+N2OEwxhhjjyXAQOY1cdJkYIhIPZdpQteW8HSwEjkixhhj7PEE1mpyqc84aTIwB+KzEZtWBGtzE/ynb2uxw2GMMcYem6Fsp8JJkwGRKwifHrgGAPi/J33hamshckSMMcbY41ONNCXnlaKkolrkaB6MkyYDsjs2A4nZxbC3NMX0njzKxBhjzDi42lrAw94SAHA1U39Hmzhp0nNyBSEqKR9bL6bhg71XAQCv9G4NB2szkSNjjDHGdEc9ryldf+c1mYodAHuwfXGZWLErHplFMvUxiQB4O/Lkb8YYY8Yl0MsehxJy9HpeEydNempfXCZe/eUi6K7jCgLm/hEDSzMJhgR5ihIbY4wxpmsBBjAZnMtzekiuIKzYFX9PwlTbil3xkCsedgVjjDFmOFTlues5xaisVogczf1x0qSHziYXaJTk7kYAMotkOJtc0HhBMcYYYw2ouZMVHKzMUCUnXMsuFjuc++KkSQ/lFD84YarPdYwxxpi+EwQBAZ7K0aZ4PS3RcdKkh9zsLHV6HWOMMWYI9L0zOCdNeijczxmeDpYQHnBeAODpYIlwP+fGDIsxxhhrUIHe+r0HHSdNeshEImDZyAAAuCdxUn2/bGQATCQPSqsYY4wxw6PaTiU+UwqFHi524qRJTw0J8sS6CZ3g4aBZgvNwsMS6CZ243QBjjDGj08rVBhamEpRVypGcXyp2OPfgPk16bEiQJwYGeOBscgFyimVws1OW5HiEiTHGmDEyNZGgg6c9YlILcSVDitbNbMUOSQMnTXrORCIgorWL2GEwxhhjjSLQS5U0FeGpEC+xw9HA5TnGGGOM6Q31vCY9nAzOSRNjjDHG9Ma/bQekINKvyeCcNDHGGGNMb7T3sIOJREBBaSWypPrVxJmTJsYYY4zpDUszE7SpmQB+JV2/SnScNDHGGGNMr9Qu0ekTTpoYY4wxplcC9HQ7FU6aGGOMMaZXVCvoeKSJMcYYY+whVCNN6YXlKCyrFDmaf3HSxBhjjDG94mBlBh9nKwD61a+JkybGGGOM6Z1AT/0r0XHSxBhjjDG9o1pBF6dHk8E5aWKMMcaY3gny5pEmxhhjjLFHUo003cwtQXmlXORolDhpYowxxpjecbO3hKutBRQEXM3Sj9EmTpoYY4wxppf0rTM4J02MMcYY00uqpCleTyaDc9LEGGOMMb2kb53BOWlijDHGmF5SjTQlZBWjSq4QORpOmhhjjDGmp1o4W8PWwhSV1Qok5ZaIHQ4nTYwxxhjTTxKJgADPmsng6eKX6DhpYowxxpjeCtCjFXSiJk3Hjx/HyJEj4eXlBUEQsH37do3zW7duxaBBg+Di4gJBEBATE3PPbchkMrz22mtwcXGBra0txo4di+zsbI1rUlJSMHz4cFhbW8PNzQ3z589HdXW1xjVHjx5Fp06dYGFhgTZt2mDjxo06frSMMcYY09a/bQfEX0EnatJUWlqKkJAQfP311w8836NHD3z00UcPvI25c+di165d2Lx5M44dO4aMjAyMGTNGfV4ul2P48OGorKzEqVOn8NNPP2Hjxo1YunSp+prk5GQMHz4cffv2RUxMDF5//XVMnToV+/fv192DZYwxxpjWVCvoYtMKsT06HVFJ+ZArSJRYBCIS557vIggCtm3bhlGjRt1z7tatW/Dz80N0dDRCQ0PVx4uKitCsWTP89ttveOaZZwAACQkJ6NChA6KiotCtWzf8/fffGDFiBDIyMuDu7g4AWL9+PRYsWIDc3FyYm5tjwYIF2LNnD+Li4tS3PX78eBQWFmLfvn11il8qlcLBwQFFRUWwt7ev/xPBGGOMMbXdsRmY+Vu0xjFPB0ssGxmAIUGej3372rx/G/ScpgsXLqCqqgoDBgxQH/P390eLFi0QFRUFAIiKikJwcLA6YQKAwYMHQyqV4sqVK+prat+G6hrVbdxPRUUFpFKpxhdjjDHGdGdfXCZm3ZUwAUBWkQyv/nIR++IyGzUeg06asrKyYG5uDkdHR43j7u7uyMrKUl9TO2FSnVede9g1UqkU5eXl973vDz/8EA4ODuovHx8fXTwkxhhjjAGQKwgrdsXjfuUw1bEVu+IbtVRn0EmTmBYtWoSioiL1V2pqqtghMcYYY0bjbHIBMotkDzxPADKLZDibXNBoMZk22j01AA8PD1RWVqKwsFBjtCk7OxseHh7qa86ePavxc6rVdbWvuXvFXXZ2Nuzt7WFlZXXf+7awsICFhYWuHgpjjDHGaskpfnDCVJ/rdMGgR5o6d+4MMzMzHDp0SH0sMTERKSkpiIiIAABERETg8uXLyMnJUV8TGRkJe3t7BAQEqK+pfRuqa1S3wRhjjLHG5WZnqdPrdEHUkaaSkhLcuHFD/X1ycjJiYmLg7OyMFi1aoKCgACkpKcjIyACgTIgA5ciQh4cHHBwcMGXKFLzxxhtwdnaGvb09Zs2ahYiICHTr1g0AMGjQIAQEBGDixIlYvXo1srKy8Pbbb+O1115TjxTNmDEDa9euxVtvvYWXX34Zhw8fxp9//ok9e/Y08jPCGGOMMQAI93OGp4Mlsopk953XJADwcLBEuJ9zo8Uk6kjT+fPnERYWhrCwMADAG2+8gbCwMHUPpZ07dyIsLAzDhw8HoGwDEBYWhvXr16tv47PPPsOIESMwduxY9OrVCx4eHti6dav6vImJCXbv3g0TExNERERgwoQJmDRpEt599131NX5+ftizZw8iIyMREhKCNWvW4LvvvsPgwYMb42lgjDHG2F1MJAKWjVRWhIS7zqm+XzYyACaSu882HL3p02TouE8TY4wxpnv74jKxYle8xqRwsfo0GfREcMYYY4wZtyFBnhgY4IGzyQXIKZbBzU5ZkmvMESYVTpoYY4wxptdMJAIiWruIHYZhr55jjDHGGGssnDQxxhhjjNUBJ02MMcYYY3XASRNjjDHGWB1w0sQYY4wxVgecNDHGGGOM1QEnTYwxxhhjdcBJE2OMMcZYHXDSxBhjjDFWB9wRXEdUW/hJpVKRI2GMMcZYXanet+uyFS8nTTpSXFwMAPDx8RE5EsYYY4xpq7i4GA4ODg+9RqC6pFbskRQKBTIyMmBnZwdB0O0mglKpFD4+PkhNTX3kDszGqKk/foCfA378TfvxA/wcNPXHDzTcc0BEKC4uhpeXFySSh89a4pEmHZFIJGjevHmD3oe9vX2T/WMB+PED/Bzw42/ajx/g56CpP36gYZ6DR40wqfBEcMYYY4yxOuCkiTHGGGOsDjhpMgAWFhZYtmwZLCwsxA5FFE398QP8HPDjb9qPH+DnoKk/fkA/ngOeCM4YY4wxVgc80sQYY4wxVgecNDHGGGOM1QEnTYwxxhhjdcBJE2OMMcZYHXDSxBhjjDFWB5w0McYYY4zVASdNeqIpdX5QKBT3Pd6UngPGGFMhIvXrX1N9HYyJiUF1dTUA/X4OOGkSWUxMDEpKSnS+ya++UigU6g0Rr169ihs3biApKQkAmsxzwNiD6PObBdMd1QdHVZIgCAJu3ryp/v+mpqysDH379sWbb74JQL+fA06aRPTOO+9g7ty52L17N+RyudG/YBKROmFavHgxxo4di+7du2PgwIFYvHixyNE1LtXvOj8/Hzk5Ofc9Z8xUjzEzMxNXr15FUVERqqqqADx4JNLYqJ6D1NRUxMfHIzU1FTKZTOSoGo/q8SclJeHGjRu4ffv2PeeMlUQiwbVr1zB//nwAwObNmxEcHIzExESRI2sctUfWAMDa2hpffPEFjh8/jgMHDogYWR0QE8WiRYvI1dWVDhw4QPn5+Rrn5HK5SFE1jtWrV5OzszNFRkbSvn376NtvvyUbGxuaNm2a2KE1qq1bt9ITTzxBPj4+NGvWLDp37pz6nEKhEDGyhqV6bNu2baOgoCDy8PCg8PBwWrZsmfpvwdj/BlTPwdatW6lt27bUoUMHatWqFb388st09uxZkaNrPJs3byYfHx/y9PSkJ598kr799lv1OWP+GyAiOnToEAmCQIMHDyYTExPauHEjERn/467tzJkzVFhYSEREGRkZNGrUKJozZw6Vl5eLHNmDcdIkgnPnzlG7du0oKiqKiIiKioro2rVr9O2339L169eJyHjeNFJTUzW+r6yspNGjR9N7772ncfzAgQNkZmZGX3zxRWOGJ5rz58+Tu7s7LVu2jNasWUOtWrWikSNH0oEDB9TXGPOL5969e8nOzo7WrFlD2dnZNHfuXPLy8qLp06dTbm4uERnP38CDHD16lOzs7OjLL78kIqLPPvuMzMzM6IcffhA5soal+nednp5OrVu3pu+++47++usvev3116lFixb0ySef3HOtsVE9rnfeeYcEQaC+fftSaWmpyFE1rmvXrpEgCBQYGEgbNmygzMxMOn78OFlZWdH+/fuJSD9fAzhpEsHly5epZcuWdPjwYbp8+TLNmjWL2rZtS35+fmRpaUmJiYlih6gTL7/8Mo0ePVrjWGlpKbVr145ef/119bHq6moiIvrPf/5Do0aNooqKCqN9sSQiun79On3yySe0YsUK9bHY2FgKDw+nESNGUGRkpPq4MTwPx48f1/g+OzubBgwYQKtWrSIiovz8fGrRogV17dqVgoKC6JVXXjHqESfVY3rzzTfp5ZdfJiKitLQ0atWqFc2YMUN9neoTuDE6deoULVy4kGbOnKl+PlJTU2np0qXk7e1t9IlTVVUVEREtX76c5s2bRyYmJjRt2jTKyMi47/XG+Bzk5+dTeHg4OTg40JIlS6hfv3508uRJWrp0KbVr144yMzPFDvG+OGlqRFevXqXS0lJKS0ujESNGUIcOHcja2ppmzJhBv//+O+Xl5VGHDh2MZrTlzp07VFlZSUREBQUF6uPLly+nTp06qUfaVBYtWkT9+vUzyhcIIuULX35+Pvn4+JClpaXGGyQR0aVLl+iJJ56gp59+mnbv3i1SlLp16NAhsrOzo9zcXPXvtbq6mn7++WeKj4+nnJwcat++vfq5mDx5Mjk6OtL48ePVI06GrHbSp/pbUH1IePXVV2nt2rVUWFioHmVTPUfbt2+nLVu2qK81JsXFxTRjxgxycHCg/v37a5xTJU6+vr73jEYbA9XvV1V+Un2/e/dudeKUlZWlvr52yd5YpKWlUXZ2NhERxcTEUEBAAK1evZo2btxINjY2NGbMGPL09KR3332XKioqRI72Xpw0NZK//vqLbG1t6c8//yQiosTERNq+fTsdOXJE/WJaWlpKTzzxBP32229ihqpz3333HXl6etK1a9eIiOjYsWPUp08fevHFF+mff/4hIuWn6oEDB9LUqVPFDLXB1E4Ejxw5Qm3atKGIiIh7XhRjY2Opbdu29NxzzxnFcH15ebn6BfLWrVvq46rHtnr1aho+fLh6ZOmLL76gDh060NixYyk9Pb3xA24AtUvUe/bsofXr1xOR8kOCj48PNW/enGbPnq0efaisrKQJEybQggUL1K8NxqD238CFCxdoxowZZGFhQZs2bdK4Li0tjebNm0cBAQGUn59vNB+iVI9j//79NHnyZBo6dChNnTqVbt68SURE+/btI1NTU5o+fTpduHCB3nvvPXJwcND4wGHorl69SiEhITR58mQ6f/48ERGtWrWKXn/9dVIoFHT27FmaPHkySSQSCgsLo5KSEpEjvhcnTY3o6aefJl9fX9qyZQvJZDL18fLycrp58yYNHz6cunTpYnSfLrOzsyk0NJSCg4Ppxo0bRKScBDxw4EByd3enLl26qM+r3iSM5UWi9ugK0b8jD4cOHSJfX1+aMGECXbx4UeNn4uLi1C+kxuLmzZskCAJ9/PHHGsfnzZtH4eHh6hfHefPm0UcffXTP4ghDVVxcTF26dKEBAwbQli1bSBAE2rx5MxERSaVSGjRoEDk6OlJxcTERKROmRYsWkbe3t9GU6WuPrqgSQyLlnJZp06aRv78//fLLLxo/k56ebhQjjXfbtm0bWVlZ0ZIlS+iTTz6hnj17kr29vXp06eDBg+Ts7EwhISHk7u6uTiyMyZdffkkTJkwgOzs7+v777+nXX3+lgQMHqqclZGdnU2RkpPq9Qt9w0tTIxo4dSy1atKDNmzdTWVkZERF98803NHToUOrevfs9Q/iG5kFzUPLy8qhTp07k7+9PSUlJRESUkJBAO3bsoMWLF9O6devUL6i1X1gNmerN4uDBg/Taa6/Riy++SCtXrlSPvOzfv598fX3pxRdfpJiYGDFDbXBVVVW0fPlyMjc31yg/f/311xQeHk5jx46liRMnko2NjXpE0hjIZDI6dOgQubm5kYWFBf38889EpEyOFAoFHTp0iAIDA8nT05P69+9PgwcPJjc3t3sSaUOl+hvYs2cPDRkyhJ588kkaNmyYeoXg9evXafr06dS+fXujG2G/W35+PnXv3p0+++wzIiJKSUkhHx8f9ei66rm6evUqHTt27J5FNIZI9ZjKyso0PgiVlZXR559/Th4eHuo5vcHBwQbxYYmTpgb09ddf05EjR+5JJEaPHk0eHh60ZcsWIlKWZH7++Wd1omSoSUPtx3n69Gnat28fxcbG0p07d4hI+aKhSpwe9CnCUJPFB9m2bRtZWlrS1KlTaeDAgdSlSxdq2bIl3b59m4iUqwbbtm1LTz31FMXGxoocre6oXizj4+PpxIkTlJKSQkREn3zyCQmCQJ9//jkRKZOKZcuW0ZgxY2jo0KEG/xzc70PDrVu3yMnJiRwdHenpp5/WOKdQKKigoIBWrlxJ8+fPp88//1z9ocJY7N69m8zMzOjNN9+k999/n3r37k1eXl7qBPLKlSv0n//8h9zc3NTTF4xRSkoKtWjRgtLS0igzM5O8vb1p+vTp6vN//PGH+rXSGKheA3bt2kUDBgwgf39/Gj58OP3888/qAYNjx47RrFmzKCIiggRBoBUrVuh9lYGTpgYUGhpKzZs3p3/++eeeF9PQ0FDq2LEjbdq0SeMfiSEmDQqFQuMxvPXWW+Tl5UW+vr5kbm5O48ePVy8hzc/Pp86dO1NAQIBRjSjcT25uLoWGhtLq1avVxy5fvkyDBg0iPz8/ysnJISLlXIaQkBCjmcOjsm3bNrK1taXWrVuThYUF/fe//6Xs7Gz69NNPSRAEWrNmDRH9m2joc28WbaSnp6vnqv3yyy80b948SkpKogMHDlDr1q1p2LBh6msN9QNSXSgUCiopKaH+/fvTwoULNc793//9H3l5edGlS5eISDnHae7cuXpbknkcZ86coYKCAiorK6Nhw4bRf//7X2rRogW98sor6t9/SkoKvfjii7R3716Ro9WtvXv3krm5OS1cuJDWr19PgwYNovDwcFqyZIk6ccrJyaFTp05R37596erVqyJH/GicNDWAX3/9VT3U3K9fP/L19aWTJ0+qEyKFQkHPPfccOTo60jPPPCNmqI/t7iHkb7/9lpo1a0bHjh2jO3fu0O7du2nQoEE0fPhw9dLz3Nxc6tKlCwUFBamX2Brj0vLk5GRyd3fX6L1UXV1NMTEx1KlTJ/rqq6/Uj9sYJn2ryOVyys/PVzcrvH79Or3//vskCAKtWrWKMjMz6dNPPyULCwv64IMPxA5XZxQKBZWXl6tXQH744YckCIK6YaNMJqPt27dTmzZtaMSIEeqf++abb+i///0vyeVyvf+U/SC1466dCMrlcgoLC1OXZGvP5YyIiKBx48apv9fHlVLaunvi8o0bN8jb25sSEhJILpfTc889R4IgaDxuIuUHTUP/4JSXl0dEyn8LcrmcSktLacSIERoJc2VlJS1evJieeOIJgx1V5KRJx+Li4igsLIw6duxIe/bsISKiPn36kK+vLx0/flz95vh///d/FB8fb9DJwn/+8x+aP38+Ef07Qvbyyy/TpEmTNK47evQode7cWX0tkXLEKSwsjJ588snGC7iRyWQy6tSpEy1ZskTjuEKhoPDwcJo5c6bGMUNXe8JvWVkZLV68WKPVxOeff66ROK1cuZKcnZ01rjEG169fJx8fHxIEgZYtW6Zxrnbi1LFjR5oxYwYJgkBXrlwRJ1gdUP3eVW+aRMreXKqVsX379qUhQ4aoz6kSp9dff10jeTR033zzDQUFBWkkPklJSeTn56eex1hcXExhYWEUGhpKH330EW3atIleeeUVsre3N+h5jb/88gv179//nn/Hffr0oTfeeIOI/n2PqK6upl69etFzzz3X6HHqAu89p0Pz58/HsmXLYGVlhbS0NMyaNQs7duzAkSNH0LZtW0yePBmjR49G9+7dcfr0abRr1w4SicRg99oaNGgQVq5cCQAoLCxUHy8uLgbw7x5ivXv3xvPPP4/vv/8ehYWFICI4Oztj1apVyM/PV2/Ya8ioZh+l6upqlJeXAwAsLCzQp08fREZGYuvWreprBUGAt7c3HB0d1Xsw6fMGlXUlCAJ27NiBUaNGoUuXLti6dStSU1PV5+fMmYPPPvsMS5cuxfr16zFlyhRcv34dTk5OIkatW1VVVXB2doaJiQnc3NyQmJiIM2fOqM9bWFhg2LBh+Pnnn9G+fXvk5OTg0qVLCAgIEDHqxyMIAvLy8jBixAh88MEH2LlzJ3r37g2pVAoAWLRoEa5fv47Zs2cDUD4HAJCXlwc7OztUV1cbxV5zAwcORHFxMV544QVkZGQAAIqKimBmZgYXFxfI5XLY2triyJEjCAgIwJ9//omPPvoI2dnZOHnyJEJCQkR+BNpT/d5KSkpQWVmJ5cuX4+rVqwCAiooKODg4ID4+HgBgYmICuVwOExMTDB48GKmpqaisrBQt9noTM2MzJj/++CM5OjrShQsXqKCggDIzM2nQoEHUpUsX2rlzJxEp+1G88sor9J///Ec9hG2II013j4r89NNPNHjwYEpPT6fNmzeTIAh04sQJjWv++OMP6tatG0mlUvWxTz75hNq2bavxCdUQ1V4hNH78eOrQoQO9+eabdOjQIZLJZDRixAjq1q0bzZkzh/7880+aOXMm2dvbG0T9Xhvnzp0je3t7mjFjBr300ktkZmZGc+bM0ejPRET04YcfkpOTk8H/3h+muLiYLl26RG3atKExY8bc08hVRTWvw1Cp4r99+za9//771Lx5c7K0tKT//e9/6mukUil99dVX5OfnR3379qV33nmHJk+eTDY2NnT58mWxQm8QycnJ1Lp1a+rRowdlZWXRqVOnyN/f/75z10pLS6m4uFijZGloav+73rRpE/Xt25dGjx6t/r3GxsaSlZUVzZ49W+O9btKkSTR27FiD7EPGSZOOLFmyhHr06EFyuVz9jyMtLY26du2q7s2kojpvLJNAv/nmG4qIiKAXXniB0tPTac6cOeTg4EB79uyhW7du0Z07d2jgwIE0YsQIjYRr69atBp84qB7Pjh07yNramt555x36+eefqU+fPuTn56d+/G+99RZ169aN2rZtSz179qTo6GhxA9exGzdu0NKlS+nDDz9UH/vmm2+oefPmtHDhwnsSJ2Mqyan+DcTExND//vc/2rRpk/rxnjlzhtq0aUPjxo2jU6dOEZGyqeXy5cs1ftYQbdq0idzd3dU9hvbt20eCIJC7u7vGvwMiZRJ57NgxGjFiBPXv35+eeeYZo0uYVFSJ06BBg2jnzp0UGhpKX375JW3ZsoWOHDlC27Ztox9++IHi4uLEDvWxHD58mJo1a6bxu/7pp5/UiZNqJexff/1FVlZW1K9fP3rppZdo8uTJZGtra7ArZTlpekyqF713332XunTpol4BpMqgDx8+TNbW1tS3b1/6/fffRYuzoW3cuJF69uxJzz77LJ09e5YWLlxIVlZW1Lx5cwoICKDQ0FCD70GlsmfPHvWqH4VCQTk5OdS7d291/5WysjJq1qwZzZ07956RxOzsbL3scvs4ioqKqEuXLuTq6kqLFy/WOLd27Vry9vamJUuWaDTsNORk4X62bNlCPj4+9MQTT1Dv3r3JzMyMdu3aRUTKzZkDAwOpW7duNGjQILK2tn7gyJMhUP3ujhw5QhERERQUFEQ5OTmUn59PO3fupA8//JD8/f3vmc9VmyGOMDyI6vlISEhQfwhMTk6mNm3aqDek7dixIwUFBVHnzp2pbdu21L59e0pISBAz7MeWkJBAb775JgUGBtJHH32kPv7TTz9Rnz59aPTo0RQfH09EytYjkyZNotGjR9OkSZMMOmHmpElHYmNjycTERP0JUmXfvn00duxY6tevHw0YMMAoVojUVvvN74cffqDevXvTc889RwUFBXTp0iXavHkzbd682eB7UKlkZWWRn5+feiI/kXLFTOfOnSkhIYFu3rxJ3t7eNG3aNPXP7Nu3j65fvy5WyI3i4sWL1LZtW3ryySfveUFct24dWVpa0ooVKwz+938/586dI2dnZ9qwYQMRKbdIEgSB3n33XXXSHBMTQ4sXL6aZM2ca9KRvImVvHSLl335UVBT16NFDnTgRKftSLVu2jPz9/TU2pf7ll1/Uk8ONJWlWPY6//vqLAgICaOnSpeqNZpOTk6lz584UHBxMSUlJVFVVRVVVVeqFEobqk08+UX/wu337Ni1YsIDat2//wMRJ9QFTlSgbesLMSZMO/fjjj2RmZkbz58+n8+fPU1JSEg0fPpxWrlxJ8fHxJAiCxg72xuLuxKlHjx703HPPqUcW7t5KxNBduHCBnnjiCZo6dSpdvnyZysvLKSAggNavX09t2rShqVOnqh9rcnIyjR8/3uj6r9zPpUuXKDQ0lKZPn35P6eG7774zir5cqrYZtW3evFm9hPzmzZvUvHlzevXVV9XnVaXI6upqg5zDWJtqA2bVajCFQkEnT56kHj16kL+/v7pUd/v2bVq+fDn5+/vTyy+/TEuWLCFBEIyyD9P+/fvJysqK1q1bp04cVZKTk6lVq1bUt29fdYNXQ3b9+nXq2bOnxrSK69evPzRxGjdunMbKQENPmDlp0rEtW7aQm5sbNW/enLy9vSksLIzKy8vp1q1b1LZtW3XWbWzuTpx69uypkTgZm4sXL1KnTp1oypQplJGRQWvXriVBEDQaFxIRLV68mIKCgoziBbMuVM/L1KlTDX5E5W6qhOHuDVQ//fRTevLJJykpKYlatGhB06dPVydHO3fupNdee81oSrK1N2Cu/aFIlTh16NBBnTilpqbSV199ReHh4RQREWE0W8OoKBQKkslkNHHiRJo3b576GJHmB0RVR/ghQ4YY/AfH6upq9T6JJ06cUP//gxKnn3/+mcLCwmjixIlGU2XhpKkBpKWlUVRUFB0/flz94rlw4ULy9/dXD90ao9pvJBs3bqRevXrRggULSCaTGfyni/u5ePEihYaG0tSpU+ngwYP05ptvkomJCa1evZpWr15Nr776KtnZ2RndpO9HuXjxIoWHh9P48eMNfqJ/bbUThtoT28+ePUu9e/cmJycneumll4jo38Uec+fOpWeffZaKiooaP+AGpNqAWdXt/kGJk0wmI7lcToWFhWKG26C6d+9Oc+fOve851Yel27dvG3yJvvZreEZGBvXt25fatGmj/kDwoMTpf//73z0LQQwZJ00NLC4ujiZOnEguLi5N4s2z9h/Wm2++ST169DCaTxj3c/HiRercuTO98sordODAAfrqq68oMDCQunbtSs8//7xBT3h8HKpEQtXx3ZioEoaPP/6YiJSdrKdOnUoeHh60du1aKikpobS0NFq4cCG5uLgY/Cqp+6m9AbNqAUTtxCk4ONhoPyDWHk0qKSmhAQMG0AsvvKA+promNTWVFi5caHQlyYsXL9L06dNp8+bN1L17d+rUqdM9iVNgYOA983uNBSdNDaiqqoouXrxI8+bNM8oXzgdRvagsX76cWrVqZdSfMomUc5y6dOlCU6dOpYyMDI3O2E2ZsT7+2gnDp59+SkTKxzp27FgKDg4mW1tb6t69O7Vu3dpoSlKP2oC5duL0zz//UFBQEHXt2tXg53DVpnoOVLs6qPor7dq1SyOJVlm0aBF16dJFPTppLD799FPq3LkznT9/nk6ePEkhISHUuXNnjcRp5syZ9MQTT1BeXp7RVRk4aWoEhr5aoD4UCgX9+eefBr01gDYuXrxITzzxBD333HPqBNnYXiyaqkclDJ988gkRKUecYmJiaOPGjXTy5ElKS0sTLeaG8KgNmFWJk1wup9OnT1NycrKo8eqS6t/A3r17adSoUdS3b18aN26cunrwxRdfkCAINGbMGJo0aRK9+OKLZG9vbxRJs+qx117x16NHDxowYAARKRtchoaGaiROSUlJRpcsqnDSxJiOGHNJqql7VMKwZs0asUNsME11A+a77dixg8zNzWnhwoU0e/ZsGjZsGFlaWqr7cR06dIgmTJhAI0eOpBkzZhjVQoh9+/bRhAkTaP/+/USknKPVqlUrWrVqFREp21CEh4dTq1atjGbRw4Nw0sSYDhlrSaqp0iZhUL2BGIv6bsCcn59vdKOslZWVNHToUFq4cKH6WFlZGb3++utkYWGh7m6tmr9pTP3IFAoFTZs2jQRBIGdnZ1q2bBndvHmTVq5cSc888wzFxsaSQqGgffv2UZ8+fYx2xbSKqdh73zFmTCwtLcUOgekA1WyiXFlZCSsrK/Tu3Rvjxo2Dk5MTlixZAltbW8ydOxcAMHnyZJSXl2P16tWYPn260WxArNqAed26dUhNTYVCocBzzz2nfnxz5swBALz11lsoLy/Hq6++ihkzZsDZ2VnMsHVux44duH79Om7duoWnnnoKgPLfh4WFBd5//33ExcXhyy+/xDfffAOJRAJAuTmtIaNam4gLgoCpU6eipKQEQUFB2LZtG7Kzs1FdXY0rV64gMjISwcHB6NevH3r27Alra2uRo29YErEDYIwxfaNKGEaNGoUuXbpg69atSE1NVZ+fM2cOPvvsMyxduhTr16/HlClTcP36daNJmADg/PnzmDRpEvz8/BAeHo6kpCT88MMPuH37tvqaOXPmYMWKFfjyyy9hampqdAnThQsX8PLLL6Nly5Zo164d9uzZg7KyMgiCAEEQYGNjAw8PD+Tl5cHMzAympspxCFXCYagEQcDhw4fx3XffAQC6dOkCFxcXJCUl4fDhw+jYsSMAICEhAW+++SaioqJgZmZm9AkTwEkTY4zdo6knDElJSdi1axcWLVqEdevW4ccff8QXX3yBv/76C+vXr9d4HhYuXIikpCS4uLiIGLHu3bhxAzt37sS0adMwbtw4DB06FLm5uVizZg1kMpk6MTI1NYWTkxOqqqpARCJHrRtyuRxnzpzB9OnTMWnSJERFReHLL79ETEwMPv/8c7z66qtYs2YNXnvtNXh5eaFZs2Zih9xoBDKW3zJjjOlAUlISNm3aBCsrKyxcuBAAsG7dOnzwwQeYMGECZsyYgZYtW6qvv3PnjlGNMEmlUvTv3x+3bt3C9OnTsXLlSvW5r7/+Gh9++CFeeuklTJkyBX5+fgA0yznGQPUc3L59Gy+++CI+++wzVFdXY9GiRTh27Bjs7OwwYMAAJCYm4q+//sLp06cRGBgodtg6Fxsbi/nz56OkpARPPPEEhgwZgvXr1+Ott95C9+7dAQCFhYVwdHQUN9DGJOJ8KsYY0ytFRUXUpUsXcnV1pcWLF2ucW7t2LXl7e9OSJUs0Jrsa26Rnoqa9AbOK6jkIDQ2lCxcuEJGyeeVPP/1EL7zwAoWHh9PYsWPVk8CNVVZWFm3atIlCQ0PJxsaG/Pz8aMmSJWKHJRoeaWKMsVqio6Px3HPPwc3NDevXr0dQUJD63Pr16zF37lwsWrQIixcvVs9hMUaxsbGYPHkywsPDMXv2bI2RlO+//x69evVC27ZtRYyw4cXGxmLixIkIDw/HrFmz1HN5AKC8vBympqYwMzMTMcLGU1VVhQULFmDt2rVwcnLCjRs3YGdnJ3ZYjY6TJsYYuwsnDErR0dGYOnUqOnXqhLlz5yIgIEDskBpd7efg9ddfN8oy3KNQrfLrwYMH0bZtW40SdVPCSRNjjN0HJwxK0dHRmDFjBlq1aoVly5bB399f7JAaHT8Hxjdvrb549RxjjN1HWFgYvvvuO8TGxuK9995DQkKC2CGJIiwsDGvXrkVmZiYcHBzEDkcU/BwYfhsFXeGRJsYYe4hz585h/vz5+N///gdPT0+xwxGNTCZr8s1b+TlgnDQxxtgj8JslYwzgpIkxxhhjrE54ThNjjDHGWB1w0sQYY4wxVgecNDHGGGOM1QEnTYwxxhhjdcBJE2OMMcZYHXDSxBhjjDFWB5w0McYYY4zVASdNjDHGGGN1wEkTY4zVIpfLoVAoxA6DMaaHOGlijOk9X19ffP755xrHQkNDsXz5chARli9fjhYtWsDCwgJeXl6YPXu2+rqKigq8+eab8Pb2ho2NDbp27YqjR4+qz2/cuBGOjo7YuXMnAgICYGFhgZSUFPj6+uKDDz7Ayy+/DDs7O7Ro0QIbNmzQiGHBggVo164drK2t0apVK7zzzjuoqqpSn1++fDlCQ0Pxww8/oEWLFrC1tcV//vMfyOVyrF69Gh4eHnBzc8PKlSs1brewsBBTp05Fs2bNYG9vj379+uHSpUu6e0IZY/ViKnYAjDH2OP766y989tln+P333xEYGIisrCyNBGPmzJmIj4/H77//Di8vL2zbtg1DhgzB5cuX0bZtWwBAWVkZPvroI3z33XdwcXGBm5sbAGDNmjV47733sHjxYmzZsgWvvvoqevfujfbt2wMA7OzssHHjRnh5eeHy5cuYNm0a7Ozs8NZbb6nvPykpCX///Tf27duHpKQkPPPMM7h58ybatWuHY8eO4dSpU3j55ZcxYMAAdO3aFQAwbtw4WFlZ4e+//4aDgwO+/fZb9O/fH9euXYOzs3NjPbWMsbsRY4zpuZYtW9Jnn32mcSwkJISWLVtGa9asoXbt2lFlZeU9P3f79m0yMTGh9PR0jeP9+/enRYsWERHRjz/+SAAoJibmnvucMGGC+nuFQkFubm60bt26B8b58ccfU+fOndXfL1u2jKytrUkqlaqPDR48mHx9fUkul6uPtW/fnj788EMiIjpx4gTZ29uTTCbTuO3WrVvTt99++8D7Zow1PB5pYowZtHHjxuHzzz9Hq1atMGTIEAwbNgwjR46EqakpLl++DLlcjnbt2mn8TEVFBVxcXNTfm5ubo2PHjvfcdu1jgiDAw8MDOTk56mP/3879g7TOxWEcf9LoEPFPQaQ4aAtidSpWFBFRQQSHGombUCi4qos4iSi4Kl1cHezgUCcHqYt1cKmDOIiTQ5E6qOgmARdL7/BioK9wb66VS4fvZ8o5kF/OyfRw8iNHR0fa29tTsViU67r6+PhQa2trVY1IJKKWlhZvHAqFZJqmAoFA1dxn3ZubG7muW7U+SXp/f1exWPybVwPghxGaANS9QCCgSqVSNffZO9TV1aW7uzvl83mdnZ1paWlJu7u7uri4kOu6Mk1T19fXMk2z6v7m5mbv2rIsGYbx5bmNjY1VY8MwvCbxy8tLJZNJbW9va2ZmRm1tbcpms0qn03+s8bu6ruuqs7Ozqu/qUzAY/DIH4N8hNAGoex0dHXp6evLGb29vur+/98aWZcm2bdm2reXlZfX39+v29lbxeFzlclkvLy8aHx//0TUVCgWFw2FtbGx4c6VSqea6g4ODen5+VkNDgyKRSM31APwcQhOAujc1NaVMJiPbthUMBrW1teWdHGUyGZXLZY2MjKipqUmHh4eyLEvhcFjt7e1KJpNKpVJKp9OKx+N6fX3V+fm5YrGYEonEt9fU29urh4cHZbNZDQ8PK5fL6fj4uOa9Tk9Pa3R0VI7jaGdnR9FoVI+Pj8rlcpqfn9fQ0FDNzwDwPfxyAEDdW19f1+TkpGZnZ5VIJOQ4jnp6eiT998lqf39fY2NjisViyufzOjk58XqCDg4OlEqltLa2pr6+PjmOo6urK3V3d9e0prm5Oa2urmplZUUDAwMqFAra3Nysea+GYej09FQTExNaXFxUNBrVwsKCSqWSQqFQzfUBfJ9R+X+jAAAAAL7gpAkAAMAHQhMAAIAPhCYAAAAfCE0AAAA+EJoAAAB8IDQBAAD4QGgCAADwgdAEAADgA6EJAADAB0ITAACAD4QmAAAAH34BdfLo3iFtDt8AAAAASUVORK5CYII=\n", - "text/plain": [ - "
" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "df = pd.DataFrame(response.json)\n", - "df = df.sort_values('username')\n", - "\n", - "df.plot(x='username', y='total_upvotes', marker='o')\n", - "plt.xticks(rotation=45, ha='right')\n", - "plt.ylabel(\"Total number of upvotes\")\n", - "plt.gca().get_legend().remove()\n", - "plt.show()" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "The total number of upvotes likely depends on the total number of posts created per user. To better assess the relative impact per user, you compare the total number of upvotes (line plot) with the total number of posts." - ] - }, - { - "cell_type": "code", - "execution_count": 14, - "metadata": {}, - "outputs": [ - { - "data": { - "text/plain": [ - "" - ] - }, - "execution_count": 14, - "metadata": {}, - "output_type": "execute_result" - }, - { - "data": { - "image/png": "iVBORw0KGgoAAAANSUhEUgAAA1cAAAHMCAYAAAA5/FJZAAAAOXRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjcuMCwgaHR0cHM6Ly9tYXRwbG90bGliLm9yZy88F64QAAAACXBIWXMAAA9hAAAPYQGoP6dpAADE60lEQVR4nOzdd3hUdfb48feUTHrvgYSEFhJKQhGkCQSkKS6uFZDq6voVdDH2/UqxLLYFQWHhZ2ddXbEgX9dFpIgiSJESBQIBQiABUkmvk8zc3x9hBmICpExyJ8l5Pc88j5m5c++ZRJI5c87nfDSKoigIIYQQQgghhGgSrdoBCCGEEEIIIURbIMmVEEIIIYQQQtiAJFdCCCGEEEIIYQOSXAkhhBBCCCGEDUhyJYQQQgghhBA2IMmVEEIIIYQQQtiAJFdCCCGEEEIIYQN6tQNoK6qqqjh06BCBgYFotZKzCiGEEK2B2WwmMzOTvn37otfL2yIhRNPIbxEbOXToEAMHDlQ7DCGEEEI0wr59+7jhhhvUDkMI0cpJcmUjgYGBQPUv5+DgYJWjEUIIIUR9pKenM3DgQOvfcSGEaApJrmzE0goYHBxMx44dVY5GCCGEEA0hLf1CCFuQ3yRCCCGEEEIIYQOSXAkhhBBCCCGEDUhyJYQQQgghhBA2IGuuhBBCCCGEXTKZTFRWVqodhmjHHBwc0Ol09T5ekishhBBCCGFXFEUhIyOD/Px8tUMRAi8vL4KCgtBoNNc9VpIrIYQQQghhVyyJVUBAAC4uLvV6UyuErSmKQmlpKVlZWQD12m5JkishhBBCCGE3TCaTNbHy9fVVOxzRzjk7OwOQlZVFQEDAdVsEZaCFEEIIIYSwG5Y1Vi4uLipHIkQ1y/+L9Vn/J8mVEEIIIYSwO9IKKOxFQ/5flORKCCGEEEIIIWxAkishhBDCDpQZqzBWmblYXIGxykypsUrtkIQQQjSQDLQQQgghVFZRaWLNj6f54OcUCsuq8HDWM3tIBA+P7IKjQ/33VxFCiIYoLS1l+vTpbNmyhaKiIvLy8vDy8lI7rFZNKldCCCGEisqMVfzjh2RWbDtJYVl1taqwrIoV207yjx+SpYIlRBO0dEU4Ozub//mf/yEsLAxHR0eCgoIYN24cu3btatbrNtbatWv56aef+Pnnn0lPT8fT01PtkK7qzJkzaDQaEhIS1A7lmqRyJYQQQqhIp9Xywc8pdT72wc8pzB3VtYUjEqJtUKMifMcdd2A0Glm7di2dO3cmMzOTbdu2cfHixWa5XlMlJycTFRVFr1691A6lzZDkSgghhGiAvIPLbXq+MmMVf3Q+Dc51P56z/zDOBtv+ufbuN9+m5xOiuSmKQlmlqd7Hm80K7/yUwoptJ633WSrCAA8Mj0Crvf4EOGcHXb0nxeXn5/PTTz/xww8/MGLECAA6derEwIEDrcecOXOGiIgIDh06RGxsrPV53t7ebN++nZEjR/LDDz8watQoNm3axDPPPMPx48cZPHgwn376KQcOHCA+Pp7z589z66238u67715zZP2XX37JwoULOXXqFMHBwTzyyCM8/vjjAIwcOZIff/wRqJ6GN2LECH744Yda55g1axb5+fls2LDBet/8+fNJSEiwHj9y5EhrgvbRRx/h4ODA//zP//DCCy+g0Wj461//yrZt29i7d2+Nc8fExHDHHXewcOFCzGYzL730Em+//TbZ2dlERUXxyiuvMH78eAAiIiIA6Nu3L0CNeN99912WLl1KSkoK4eHhPProozz88MMAGI1G4uPj+fLLL8nLyyMwMJCHHnqIZ5999po/z8aS5EoIIYRQkUGvw9FBS0WludZjjg5aDHpZcyVEWaWJ6IXf1etYH1cDO58edc2K8J9HdGbYq9vJLTFe81yJL4zDpZ4fbri5ueHm5saGDRu48cYbcXR0rNfzrmbx4sWsXLkSFxcX7r77bu6++24cHR355JNPKC4u5vbbb+ett97i6aefrvP5Bw4c4O6772bx4sXcc889/Pzzzzz88MP4+voya9Ys1q9fzzPPPMORI0dYv349BoOhSfGuXbuW+++/n3379rF//34efPBBwsLCeOCBB5g2bRovv/wyycnJdOnSBYCjR4/y22+/8eWXXwKwYsUKli5dyv/7f/+Pvn378v7773Pbbbdx9OhRunXrxr59+xg4cCBbt26lZ8+e1ng//vhjFi5cyMqVK+nbty+HDh3igQcewNXVlZkzZ/Lmm2/y9ddf89lnnxEWFkZaWhppaWlNeq3XIsmVEEIIoSJFUYjt6M3elNptQ7EdvTErCjpkvx8h6svfzZGLxUbrGsbfKyyrIrfEiL+b43WTq4bQ6/V8+OGHPPDAA6xZs4Z+/foxYsQI7r33Xvr06dPg87300ksMHToUgPvvv59nn32W5ORkOnfuDMCdd97J9u3br5pcLVu2jNGjR7NgwQIAunfvTmJiIq+//jqzZs3Cx8cHFxcXDAYDQUFBjXzVl4WGhvLGG2+g0WiIjIzk8OHDvPHGGzzwwAP07NmTmJgYPvnkE2s8H3/8MYMGDaJr1+rW57///e88/fTT3HvvvQC8+uqrbN++neXLl7Nq1Sr8/f0B8PX1rRHvokWLWLp0KX/84x+B6gpXYmIi/+///T9mzpxJamoq3bp1Y9iwYWg0Gjp16tTk13otklwJIYQQKtLrtNwQ7o2Cwq/n8qmoNOPooCW2ozc3hHuj08nsKSGcHXQkvjCu3sfrtVo8nPV1JlgeznoC3J34au6Qel23Ie644w5uueUWfvrpJ/bs2cO3337La6+9xrvvvsusWbMadK4rE7LAwEBcXFysiZXlvn379l31+ceOHeMPf/hDjfuGDh3K8uXLMZlM6HS2rYrfeOONNVooBw8ezNKlS63XmjZtGu+//z4LFixAURT+/e9/Ex8fD0BhYSEXLlywJpNXxvvrr79e9ZolJSUkJydz//3388ADD1jvr6qqsg7nmDVrFjfffDORkZGMHz+eW2+9lbFjx9rypdcgyZUQQgihsmJjFYHuTtw/rDPlRhNOBh2lRpMkVkJcotFo6t2eB9VrGWcPiaix5spi9pAIqszmBp2vIZycnLj55pu5+eabWbBgAX/6059YtGgRs2bNQqut/jetKIr1+MrKyjrP4+DgYP1vjUZT42vLfWZz7XZiW9NqtTXihavHfC1Tpkzh6aef5uDBg5SVlZGWlsY999zTpNiKi4sBeOeddxg0aFCNxyzJY79+/UhJSeHbb79l69at3H333YwZM4YvvviiSde+GkmuhBBCCJWdzCxm16kcugW44e1i4PCFAvp08OLGLr5qhyZEq+Rs0PPwyOq1PWrvHxcdHW0dBmFpbUtPT7cOZmiu0eJRUVG1RsDv2rWL7t27N6hq5e/vz5EjR2rcl5CQUCvZ+/2wij179tCtWzfrtTp27MiIESP4+OOPKSsr4+abbyYgIAAADw8PQkJC2LVrl3UYiCVey0AQyxork+nyYJPAwEBCQkI4ffo006ZNu+pr8PDw4J577uGee+7hzjvvZPz48eTm5uLj41Pv70N9SXIlhBBCqOxUVvWnr6E+LpjMCmVGE9nFFSpHJUTr5uig488jOjN3VFeKyitxd3KgymxutsTq4sWL3HXXXcyZM4c+ffrg7u7O/v37ee2116ztec7Oztx444288sorREREkJWVxXPPPdcs8Tz++OPccMMNvPjii9xzzz3s3r2blStX8o9//KNB54mLi+P111/nn//8J4MHD+Zf//oXR44csSaHFqmpqcTHx/PnP/+ZgwcP8tZbb7F06dIax0ybNo1FixZhNBp54403ajz25JNPsmjRIrp06UJsbCwffPABCQkJfPzxxwAEBATg7OzMpk2b6NixI05OTnh6evL888/z6KOP4unpyfjx46moqGD//v3k5eURHx/PsmXLCA4Opm/fvmi1Wj7//HOCgoKabbNkSa6EEEIIFRWVV5JZWA4a6OLvZl1gnyPJlRBNZmn983WrntxnoPlabd3c3Bg0aBBvvPEGycnJVFZWEhoaygMPPMBf//pX63Hvv/8+999/P/379ycyMpLXXnutWdYA9evXj88++4yFCxfy4osvEhwczAsvvNDgtV/jxo1jwYIFPPXUU5SXlzNnzhxmzJjB4cOHaxw3Y8YMysrKGDhwIDqdjr/85S88+OCDNY658847mTdvHjqdjsmTJ9d47NFHH6WgoIDHH3+crKwsoqOj+frrr+nWrRtQPTDkzTff5IUXXmDhwoUMHz6cH374gT/96U+4uLjw+uuv8+STT+Lq6krv3r2ZP38+AO7u7rz22mucPHkSnU7HDTfcwMaNG60tmramUX7fRCka5dy5c4SGhpKWlkbHjh3VDkcIIUQzsfU+Vwmpefx4IpsOXs7cOSD00sanyQA8NLILjs0wil32ubpM/n7bn/LyclJSUoiIiMDJyUntcEQ9jBw5ktjYWJYvX652KM2iIf9PykpZIYQQQkWWlsCuAW5AdSuTu1P1WoaLRbYbEy2EEKL5SXIlhBBCqKS0oorzBWUAdLmUXAH4u1Uv3M4uKlclLiGEEI0ja66EEMJOlRmr0Gm1NRZiN9foYKGO09kloECgh5O1WgXg5+7I6ZwSsoulciWEsH8//PCD2iHYDfkrLYQQdqh63c1p1UcIi+Z1MrsIuNwSaOF/afG9DLUQQojWRZIrIYSwM2XGKtb8eLrG5peFZVXWr/88orNUsNqAikoT53KrWwK7+v8uuXK/nFyZzQparabF4xNCCNFwsuZKCCHsjE6r5YOfU+p87IOfU9A30/hY0bJOZ5dgVhR83Qx4uRpqPObh7ICDTovJrJBfKq2BQgjRWshfaCGEsDNF5ZUUllXV+VhhWRVF5ZUtHJFoDqeyL00J9Hev9ZhGo8HPMtRCWgOFEKLVkORKCCHsjLuTAx7Odbf9eTjraww+EK1TZZWZsxdLgNrrrSysrYEyjl0IIVoNSa6EEMLOmMxmZg+JqPOx2UMiqDKbWzgiYWtnLpZgMit4ujhYK1S/53dpqIVUroQQ9XXmzBk0Gg0JCQlqh2J1/PhxbrzxRpycnIiNjVU7nGYnK6KFEMLOOBv0/PmmzpgVhbW7z1inBc4cHC7TAtsI68bB/m6gqXtYhaVyJcmVEJflHVzeYtfy7je/wc+ZNWsWa9eu5eWXX+aZZ56x3r9hwwZuv/12FEWxYYStw6JFi3B1dSUpKQk3t7or9c3thx9+YNSoUeTl5eHl5dWs15LKlRBC2KHXNiXRu4Mn+/46hgPPjWHPs6Pp1cGT0zklaocmmqjKZCYl59otgQC+bo6gqd5ouNRY9xo8IYT9cXJy4tVXXyUvL0/tUGzGaGx8e3JycjLDhg2jU6dO+Pr62jAq+yTJlRBC2JnMwnI+2nuWBz86wLncMnzdHHnxm2P8+aMDfL7/nNrhiSZKyy2l0mTGzVFPoIfTVY9z0Gnxcq5uGcwpkuqVEK3FmDFjCAoK4uWXX77qMYsXL67VIrd8+XLCw8OtX8+aNYvJkyezZMkSAgMD8fLy4oUXXqCqqoonn3wSHx8fOnbsyAcffFDr/MePH2fIkCE4OTnRq1cvfvzxxxqPHzlyhAkTJuDm5kZgYCDTp08nJyfH+vjIkSOZN28e8+fPx8/Pj3HjxtX5OsxmMy+88AIdO3bE0dGR2NhYNm3aZH1co9Fw4MABXnjhBTQaDYsXL67zPJbrzZs3D09PT/z8/FiwYEGNSl9eXh4zZszA29sbFxcXJkyYwMmTl7csOXv2LJMmTcLb2xtXV1d69uzJxo0bOXPmDKNGjQLA29sbjUbDrFmzAPjiiy/o3bs3zs7O+Pr6MmbMGEpKmvYhpiRXQghhZz77JQ2TWeGGcG+6BlZXNuJ6BADwzW8XMJnbX1tJW2JpCezi74bmKi2BFrKZsBCtj06nY8mSJbz11lucO9e0D8S+//57Lly4wI4dO1i2bBmLFi3i1ltvxdvbm7179/LQQw/x5z//udZ1nnzySR5//HEOHTrE4MGDmTRpEhcvXgQgPz+fuLg4+vbty/79+9m0aROZmZncfffdNc6xdu1aDAYDu3btYs2aNXXGt2LFCpYuXcrf//53fvvtN8aNG8dtt91mTXrS09Pp2bMnjz/+OOnp6TzxxBNXfa1r165Fr9ezb98+VqxYwbJly3j33Xetj8+aNYv9+/fz9ddfs3v3bhRFYeLEiVRWVk/QnTt3LhUVFezYsYPDhw/z6quv4ubmRmhoKF9++SUASUlJpKens2LFCtLT05kyZQpz5szh2LFj/PDDD/zxj39scuumJFdCCGFHTGaFf+9LBWDqoDDr/SO6++Pp7EBWUQV7Uy6qFZ5oIrNZsbZ2Xqsl0MLf/dI4dqlcCdGq3H777cTGxrJo0aImncfHx4c333yTyMhI5syZQ2RkJKWlpfz1r3+lW7duPPvssxgMBnbu3FnjefPmzeOOO+4gKiqK1atX4+npyXvvvQfAypUr6du3L0uWLKFHjx707duX999/n+3bt3PixAnrObp168Zrr71GZGQkkZGRdcb397//naeffpp7772XyMhIXn31VWJjY1m+fDkAQUFB6PV63NzcCAoKuuaaq9DQUN544w0iIyOZNm0ajzzyCG+88QYAJ0+e5Ouvv+bdd99l+PDhxMTE8PHHH3P+/Hk2bNgAQGpqKkOHDqV379507tyZW2+9lZtuugmdToePjw8AAQEBBAUF4enpSXp6OlVVVfzxj38kPDyc3r178/DDDzd5XZgkV0IIYUd+SMriQkE5Xi4OTOgVbL3foNcyoVcQAF8nXFArPNFE5/PLKK804WzQEeLlfN3jL08MlHHsQn1paWmMHDmS6Oho+vTpw+eff652SHbt1VdfZe3atRw7dqzR5+jZsyfaKzaODwwMpHfv3tavdTodvr6+ZGVl1Xje4MGDrf+t1+sZMGCANY5ff/2V7du34+bmZr316NEDqF4fZdG/f/9rxlZYWMiFCxcYOnRojfuHDh3aqNd844031qjmDx48mJMnT2IymTh27Bh6vZ5BgwZZH/f19SUyMtJ6rUcffZSXXnqJoUOHsmjRIn777bdrXi8mJobRo0fTu3dv7rrrLt555x2brJOT5EoIIezIx3urq1Z39e+I0++mAt4WEwLAt0cyMFbJOPbWyNIS2NnPDa322i2BAH6XJgbmlRgxyQh+oTK9Xs/y5ctJTExk8+bNzJ8/v8nrU9qym266iXHjxvHss8/Wekyr1dZqP7O0t13JwaHmvoYajabO+8wN+P1QXFzMpEmTSEhIqHE7efIkN910k/U4V1fXep/THvzpT3/i9OnTTJ8+ncOHDzNgwADeeuutqx6v0+nYsmUL3377LdHR0bz11ltERkaSkpLSpDgkuRJCCDtxLq+U7UnVnz5OGRhW6/FBnX0JcHekoKySHSeyWzo80USKopCcfWkEez1aAgHcHfU4OegwKwq5Ur0SKgsODrYOYQgKCsLPz4/c3Fx1g7Jzr7zyCv/5z3/YvXt3jfv9/f3JyMiokWDZcm+qPXv2WP+7qqqKAwcOEBUVBUC/fv04evQo4eHhdO3atcatIQmVh4cHISEh7Nq1q8b9u3btIjo6usEx7927t9Zr6NatGzqdjqioKKqqqmocc/HiRZKSkmpcKzQ0lIceeoj169fz+OOP88477wBgMFS3WJtMphrX0Gg0DB06lOeff55Dhw5hMBj46quvGhz7lSS5EkIIO7HulzQUBYZ08aWzf+033zqthlv6VLcKfv2rtAa2NhkF5ZRUVGHQawn1uX5LIAAajWwmbEd27NjBpEmTCAkJQaPRWNd6XI3JZGLBggVERETg7OxMly5dePHFF2u8oV68eDEajabGzdKipUbsq1atIjw8HCcnJwYNGsS+ffvqPO7AgQOYTCZCQ0NtHmtb0rt3b6ZNm8abb75Z4/6RI0eSnZ3Na6+9RnJyMqtWreLbb7+12XVXrVrFV199xfHjx5k7dy55eXnMmTMHqB78kJuby5QpU/jll19ITk7mu+++Y/bs2bWSj+t58sknefXVV1m3bh1JSUk888wzJCQk8Je//KXBMaemphIfH09SUhL//ve/eeutt6zn6datG3/4wx944IEH2LlzJ7/++iv33XcfHTp04A9/+AMA8+fP57vvviMlJYWDBw+yfft2a0LZqVMnNBoN33zzDdnZ2RQXF7N3716WLFnC/v37SU1NZf369WRnZ1uf01iSXAkhhB2oNJn59Jc0AKYN6nTV4yytgVsSM2Xvo1bG0hIY4eeKTlv/P7+WiYEy1EJ9JSUlxMTEsGrVqnod/+qrr7J69WpWrlzJsWPHePXVV3nttddqtSr17NmT9PR06+33wwl+b9euXXW2kCUmJpKZmdno2NetW0d8fDyLFi3i4MGDxMTEMG7cuFrreXJzc5kxYwZvv/32NeMU1V544YVabXtRUVH84x//YNWqVcTExLBv375rTtJrqFdeeYVXXnmFmJgYdu7cyddff42fnx+AtdpkMpkYO3YsvXv3Zv78+Xh5edVY31Ufjz76KPHx8Tz++OP07t2bTZs28fXXX9OtW7cGxzxjxgzKysoYOHAgc+fO5S9/+QsPPvig9fEPPviA/v37c+uttzJ48GAURWHjxo3WNkmTycTcuXOJiopi/PjxdO/enX/84x8AdOjQgeeff55nnnmGwMBA5s2bh4eHBzt27GDixIl0796d5557jqVLlzJhwoQGx34ljdIet4puBufOnSM0NJS0tDQ6duyodjhCiBaUd3B5k89xKrOI/x5Ox8WgZ86wCHRXW4+jKHz48xkKyioZ3yuIyCCPJl8bwLvffJucpz1o1M9bUfjg5zMUllVya59gugS41/upiRcK2JKYSUdvZ+7ob5sqgfy8L7P8/e7atSsODg7MnTuXuXPnXvd5Go2Gr776ismTJ1/1mFtvvZXAwEDrlDaAO+64A2dnZ/71r38B1ZWrDRs21LslzGw2069fP7p168ann36KTle9NjMpKYkRI0YQHx/PU0891ajYBw0axA033MDKlSut1woNDeWRRx7hmWeeAaCiooKbb76ZBx54gOnTp9cr5oYqLy8nJSWFiIgInJyuvhecaDtGjhxZY8qgvWnI/5NSuRJCCDtw+HwBAD1DPK6eWAFoNEQGVr8xP5FR1BKhCRvILqqgsKwSvU5DmG/DFon7u18xMVA+D20227dvJzExsV6JVX0NGTKEbdu2Wcdb//rrr+zcubPWJ+MnT54kJCSEzp07M23aNFJTU696Tq1Wy8aNGzl06BAzZszAbDaTnJxMXFwckydPvm5idTVGo5EDBw4wZsyYGtcaM2aMdb2QoijMmjWLuLi4ZkushGjtJLkSQgiVFZQaSc0tBQ306uB53eMjg6qTqzO5pZRXNqw/Xqjj1KVBFp18XHHQNexPr4+rAa1GQ0WliaIKaQVtTZ555hnuvfdeevTogYODA3379mX+/PlMmzbNesygQYP48MMP2bRpE6tXryYlJYXhw4dTVHT1D09CQkL4/vvv2blzJ1OnTiUuLo4xY8awevXqRseak5ODyWQiMDCwxv2BgYFkZGQA1e2I69atY8OGDcTGxhIbG8vhw4cbfU0h2iK92gEIIUR7Z6lahfu44uHscJ2jwcfNET83R3KKK0jOKqZnPRIyoS7Leqtu9ZwSeCWdVouPq4Gc4gpyiipwd7r+/yPCPnz22Wd8/PHHfPLJJ/Ts2ZOEhATmz59PSEgIM2fOBKhRxerTpw+DBg2iU6dOfPbZZ9x///1XPXdYWBgfffQRI0aMoHPnzrz33ns19ghqDsOGDWvQyG8h6uuHH35QOwSbkcqVEEKoyGQ2c/RCIQC9O9Y/Sep+qTUwKVNaA+1dXkkFuSVGtFoN4X6N2zfGT4ZatEpPPvmktXrVu3dvpk+fzmOPPcbLL7981ed4eXnRvXt3Tp06dc1zZ2Zm8uCDDzJp0iRKS0t57LHHmhSrn58fOp2u1kCMzMxMgoKCmnRuIdoTSa6EEEJFp7KKKa804eakJ7wBa3EsrYFpeaWUSKuYXbNUrcK8XXD83cbQ9WVZd5Uj49hbldLS0lrT13Q63TWrP8XFxSQnJxMcHHzVY3Jychg9ejRRUVGsX7+ebdu2sW7duiZNmzMYDPTv359t27ZZ7zObzWzbto3Bgwc3+rxNITPXhL1oyP+L0hYohBAqsrQE9grxRHutQRa/4+HsQLCnE+kF5ZzILKJvmHdzhSia6FQDNw6ui3Ucu2wkrKri4uIaFaWUlBQSEhLw8fEhLCyMlStX8tVXX1kTlEmTJvG3v/2NsLAwevbsyaFDh1i2bJl1vyGAJ554gkmTJtGpUycuXLjAokWL0Ol0TJkypc4YzGYzEyZMoFOnTqxbtw69Xk90dDRbtmwhLi6ODh061FnFul7sAPHx8cycOZMBAwYwcOBAli9fTklJCbNnz7bJ96++LKO1S0tLcXau555wQjSj0tJS4PL/m9ciyZUQQqgkt7iC83llaDQaeoY0fN1UZKA76QXlJElyZbcKyyrJKqxAo4HO/o1rCQTwczcAkF9mpLLKjINeGk/UsH//fkaNGmX9Oj4+HoCZM2fy4YcfkpOTQ3JysvXxt956iwULFvDwww+TlZVFSEgIf/7zn1m4cKH1mHPnzjFlyhQuXryIv78/w4YNY8+ePfj7+9cZg1arZcmSJQwfPhyDwWC9PyYmhq1bt171edeLHeCee+4hOzubhQsXkpGRQWxsLJs2bao15KK56XQ6vLy8rPtrubi4NPt6MiHqoigKpaWlZGVl4eXlZd364FpknysbkX2uhGi/GrvP1Y9JWSSk5dPF341bL20O3BClFVW8u/M0igKzhoTj6WK4/pOuQvY9qr+G/LwPnc1jx8lsm+xR9e5PpympqOLuAaEEezXt03z5eV8mf7/tk6IoZGRkkJ+fr3YoQuDl5UVQUFC9knypXAkhhAqqTGYS0xs+yOJKLo56Qr1dSM0t5URmETdE+NoyRGED1pZA/8a3BFr4uzlSUlFFTnFFk5MrIeydRqMhODiYgIAAKisr1Q5HtGMODg71qlhZSHIlhBAqOJFZhLHKjIezA2E+Lo0+T2SgO6m5pSRJcmV3SiqquFBQBkCXJqy3svBzd+TMxRKZGCjaFZ1O16A3tkKoTZq2hRBCBZZBFr07eDZpLUGXADd0Wg0Xi41clElydiU5uxgUCPJ0ws0Ge1NdHmohP2chhLBXklwJIUQLyy4sJ6OgHK1WQ3SIR5PO5eigs45wP54he17Zk+Qs27UEAvhfGmpxsdgoI6qFEMJOSXIlhBAtzFK16urvhouh6d3Zlj2vTmQWgbzptgvlRhNpedUtgU0ZwX4lT2cDOq2GSpOZgjJZgyKEEPZIkishhGhBxiqztcLUu0PjBln8XoSfKw46LYVllaQXlNvknKJpTucUoygKfm6OTZrieCWtVoPfpdbAHFl3JYQQdkmSKyGEaEFJGYVUmsx4uxro6G2biW96nZYul/ZQOpEprYH2wNoSaKOqlYV13ZUkV0IIYZckuRJCiJaiKPx2xSALbLgpZmRQ9dqtE5nFmM3SGqgmY5WZs7mlgO2TKz93GWohhBD2TJIrIYRoIRmF5eQUVaDTaogKbtogi98L83HByUFHqbGKc3mlNj23aJgzOSWYzApeLgZ8XW3TEmhhqVzlSHIlhBB2SZIrIYRoIYfPVVetuge64+Rg231btFoN3S5VSZJkaqCqamwcbMPqJIDvpYmBReVVlFeabHpuIYQQTadqcrVjxw4mTZpESEgIGo2GDRs21Hh88eLF9OjRA1dXV7y9vRkzZgx79+6tcUxubi7Tpk3Dw8MDLy8v7r//foqLi2sc89tvvzF8+HCcnJwIDQ3ltddeqxXL559/To8ePXBycqJ3795s3LjR5q9XCNF+VVSaSMq07SCL37NMDTyVXYzJbG6Wa4hrqzKZOZNTAti+JRDAUa/Dw7l6zywZaiGEEPZH1eSqpKSEmJgYVq1aVefj3bt3Z+XKlRw+fJidO3cSHh7O2LFjyc7Oth4zbdo0jh49ypYtW/jmm2/YsWMHDz74oPXxwsJCxo4dS6dOnThw4ACvv/46ixcv5u2337Ye8/PPPzNlyhTuv/9+Dh06xOTJk5k8eTJHjhxpvhcvhGhXjqUXYjJXT48L9nRqlmuEeDnj5qjHWGXmTI60BqohNbeUSpMZNyc9gR6OzXIN2UxYCCHsV9M3WGmCCRMmMGHChKs+PnXq1BpfL1u2jPfee4/ffvuN0aNHc+zYMTZt2sQvv/zCgAEDAHjrrbeYOHEif//73wkJCeHjjz/GaDTy/vvvYzAY6NmzJwkJCSxbtsyahK1YsYLx48fz5JNPAvDiiy+yZcsWVq5cyZo1a5rp1Qsh2g1Fse5t1bujbQdZXEmj0dA90J2DqXkkZRbRpRkqJ+LaTmU1X0ughZ+bI8nZxVK5EkIIO9Rq1lwZjUbefvttPD09iYmJAWD37t14eXlZEyuAMWPGoNVqre2Du3fv5qabbsJguLyoeNy4cSQlJZGXl2c9ZsyYMTWuN27cOHbv3n3VeCoqKigsLLTeiopkjYMQom7n88vILTHioNPS41LrXnOxtAam5BRjrJLWwJZkNiuczm6eEexX8peJgUIIYbfsPrn65ptvcHNzw8nJiTfeeIMtW7bg5+cHQEZGBgEBATWO1+v1+Pj4kJGRYT0mMDCwxjGWr693jOXxurz88st4enpab9HR0U17oUKINssyyCIyyB2D3raDLH4vwN0RLxcDVabLb/RFyziXV0pFlRkXg44QL9vsYVYXS3KVW2KUsftCCGFn7D65GjVqFAkJCfz888+MHz+eu+++m6ysLLXD4tlnn6WgoMB6S0xMVDskIYQdKjNWcfJSktNcgyxq0GiIDKyuXsnUwJZlaQns7O+GpplaAgE8nPQY9FpMZoW8UmOzXUcIIUTD2X1y5erqSteuXbnxxht577330Ov1vPfeewAEBQXVSrSqqqrIzc0lKCjIekxmZmaNYyxfX+8Yy+N1cXR0xMPDw3pzd2/eVh8hROuUeKEQs1kh0MOJAI/mGWTxe5bWwNTcUsqNMq67JSiKQnJ2800JrEGjwc8y1ELWXQkhhF2x++Tq98xmMxUV1X9MBg8eTH5+PgcOHLA+/v3332M2mxk0aJD1mB07dlBZWWk9ZsuWLURGRuLt7W09Ztu2bTWus2XLFgYPHtzcL0cI0YYpisLhC5cGWbRE1eoSb1cD/u6OmBWFk1lSvWoJ6QXllBqrMOi1dPR2afbryWbCQghhn1RNroqLi0lISCAhIQGAlJQUEhISSE1NpaSkhL/+9a/s2bOHs2fPcuDAAebMmcP58+e56667AIiKimL8+PE88MAD7Nu3j127djFv3jzuvfdeQkJCgOqJgwaDgfvvv5+jR4+ybt06VqxYQXx8vDWOv/zlL2zatImlS5dy/PhxFi9ezP79+5k3b16Lf0+EEG1HWm4pBaWVGPRaujfzIIvfs7YGZkpy1RJOZVa3BHbxc0Onbb6WQAvrUAupXAkhhF1RNbnav38/ffv2pW/fvgDEx8fTt29fFi5ciE6n4/jx49xxxx10796dSZMmcfHiRX766Sd69uxpPcfHH39Mjx49GD16NBMnTmTYsGE19rDy9PRk8+bNpKSk0L9/fx5//HEWLlxYYy+sIUOG8Mknn/D2228TExPDF198wYYNG+jVq1fLfTOEEG2OZZBFVJAHDrqW/XVrSebO55dRXF55naNFkygKpy6tq2up8fd+V+51pchQCyGEsBeq7nM1cuRIlGv8UVi/fv11z+Hj48Mnn3xyzWP69OnDTz/9dM1j7rrrLmtFTAghmqqkoorknOo1OL07tlxLoIW7kwMhXs5cyC/jRGYx/Tp5t3gM7UVWUQVF5ZXodVo6+TZ/SyCAr5sBjQbKjCZKjCZcHVX9cy6EEOKSVrfmSgghWoOj5wtQFIUQL2d8L1UZWpq0BrYMy5TACF8X9C1UodTrtHi5VO/fKJsJCyGE/ZDkSgghbExRFI6oMMji97oFVo8EzyosJ19GdjcPRbEmVy3VEmjh7yabCQshhL2R5EoIIWzsTE4JReVVODnomn8s9zU4G/SE+VS3qcmeV80jt8RIXqkRnVZDhJ9ri15bhloIIYT9keRKCCFs7PD56qpVdLBHi7WJXU2N1kAZfGBzlqpVmI8LBr2uRa/tJ+PYhRDC7khyJYQQNlRYVknKxUuDLFRsCbToEuCKTqshr8QoFY5mYJkS2NW/5SuUlspVXqmRKpO5xa8vhBCiNkmuhBDCho5eKAAFQn1c8HI1qB0OBr3O2q4mgy1sq6C0kuyiCjQaDREqJFeuBh3OBh2KUt2eKIQQQn2SXAkhhI2YzQpHLhQC9lG1srC0Bp7ILLrm9heiYZKzq5PVjt7OOBtatiUQAI3m8lALqUoKIYRdkORKCCFs5HR2MaUVVbgY9HRWoZJxNeF+rhj0WorKq0gvKFc7nDbDst5KjZZACz9JroQQwq5IciWEEDZiGWTRM8QDnVajcjSX6XVaulxKAGRqoG0UX5GotvQI9itZ1l3JUAshhLAPklwJIYQNFJQaSc0tBQ30sqOWQIvIoOrWwJNZRZjN0hrYVKcvDbII9nTC1VGvWhzWcezFFTINUggh7IAkV0IIYQOWqlW4jysezg4qR1NbqLcLzgYdZUYTabmlaofT6p20tAQGuKsah7eLAZ1Wg7HKTGF5laqxCCGEkORKCCGazGQ2c9QyyKKj/VWtALRaDd0CrtjzSjRamdHE+fzqBLWLymvrtFoNPpemUsq6KyGEUJ8kV0II0USnsooprzTh5qQn3NdV7XCuytIamJxdLPsiNUFKdjGKUt2S5+mifpVSNhMWtpCWlsbIkSOJjo6mT58+fP7552qHJESrJMmVEEI0kaUlsFeIJ1o7GmTxe8GeTrg76TFWmTmTU6J2OK3WSTuYEngl67orqVyJJtDr9SxfvpzExEQ2b97M/PnzKSmR3xNCNJQkV0II0QQnM4s4n1eGRqOhZ4h9tgRaaDQaugdKa2BTFJZXkpZX3RLYVcUpgVey7nUllSvRBMHBwcTGxgIQFBSEn58fubm56gYlRCskyZUQQjTBx3tTAejs54qbk3pT4+rL0hqYklNCRZVJ5Whan+3HszCZFbxdDda1Tmrzu1S5KiyrxCg/02a1Y8cOJk2aREhICBqNhg0bNlzzeJPJxIIFC4iIiMDZ2ZkuXbrw4osv2nwz7/rGtWrVKsLDw3FycmLQoEHs27evzuMOHDiAyWQiNDTUpnEK0R5IciWEEI1UZjTx5cFzgP0Osvg9fzdHvF0NmMwKp7Ok5aehNh3JAC61BGrsowXUyUFnTexzio0qR9O2lZSUEBMTw6pVq+p1/Kuvvsrq1atZuXIlx44d49VXX+W1117jrbfeuupzdu3aRWVlZa37ExMTyczMbHRc69atIz4+nkWLFnHw4EFiYmIYN24cWVlZNY7Lzc1lxowZvP322/V6jUKImiS5EkKIRvrPbxcoKq/Cw9mBMB8XtcOpH42GSGkNbJQyo4kfkrIB+2kJtLC0BubIuqtmNWHCBF566SVuv/32eh3/888/84c//IFbbrmF8PBw7rzzTsaOHXvVipHZbGbu3LlMnToVk+lyFTIpKYm4uDjWrl3b6LiWLVvGAw88wOzZs4mOjmbNmjW4uLjw/vvvW4+pqKhg8uTJPPPMMwwZMqRer1EIUZMkV0II0UifXGoJ7N3BE42dVDHqw9IamJpbSplR9kaqrx0nsymrNOHu5EDApVY8e2FddyXJVaONGjWK6Ojoelel6mPIkCFs27aNEydOAPDrr7+yc+dOJkyYUOfxWq2WjRs3cujQIWbMmIHZbCY5OZm4uDgmT57MU0891ag4jEYjBw4cYMyYMTWuNWbMGHbv3g2AoijMmjWLuLg4pk+f3qjrCCHA/hcICCGEHTpyvoCEtHwcdBqiQzzUDqdBvFwMBHo4kVlYzsnMYvqEeqkdUqtgbQkMsJ+WQAvLuisZatF427dvp2PHjjY95zPPPENhYSE9evRAp9NhMpn429/+xrRp0676nJCQEL7//nuGDx/O1KlT2b17N2PGjGH16tWNjiMnJweTyURgYGCN+wMDAzl+/DhQ3Y64bt06+vTpY12z9dFHH9G7d+9GX1eI9kiSKyGEaIRP9lVXrcb1DMLFcELlaBouMtCdzMJykjKLJLmqB2OVma3Hqte72FtLIFyuXF0sqUBRlFZVSW3LPvvsMz7++GM++eQTevbsSUJCAvPnzyckJISZM2de9XlhYWF89NFHjBgxgs6dO/Pee+81+8902LBhmM2y/50QTSVtgUII0UDFFVX836HzAEwb1EnlaBqnW6A7aOBCfhlF5bUXz4uafk7Ooai8Cn93R4I9ndQOpxZPFwf0Oi1VJoX8Uvl52osnn3ySZ555hnvvvZfevXszffp0HnvsMV5++eVrPi8zM5MHH3yQSZMmUVpaymOPPdakOPz8/NDpdLUGYmRmZhIUFNSkcwshapLkSgghGmjDofOUGE109nflxs4+aofTKG5Oejp4OQNwIkMGW1zPd0erWwLHRgfaZVVIo9Hg51Y9Gl7WXdmP0tJStNqab7V0Ot01K0Q5OTmMHj2aqKgo1q9fz7Zt21i3bh1PPPFEo+MwGAz079+fbdu2We8zm81s27aNwYMHN/q8QojapC1QCCEaQFEU695W0wZ1sss32vUVGejO+bwykjKLGHP9w9stk1lh89HqT/zH9woCO81F/d0cySgoJ7u4gu64qx1Om1RcXMypU6esX6ekpJCQkICPjw9hYWGsXLmSr776yprETJo0ib/97W+EhYXRs2dPDh06xLJly5gzZ06d5zebzUyYMIFOnTqxbt069Ho90dHRbNmyhbi4ODp06FBnFet6cQHEx8czc+ZMBgwYwMCBA1m+fDklJSXMnj3blt8iIdo9Sa6EEKIBEtLyOZZeiEGv5Y5+HdQOp0m6BbjzQ1I22UUVnMoqtsu1RPbglzO5XCwx4unswI2dfSn+Ve2I6mYZaiHj2JvP/v37GTVqlPXr+Ph4AGbOnMmHH35ITk4OycnJ1sffeustFixYwMMPP0xWVhYhISH8+c9/ZuHChXWeX6vVsmTJEoYPH47BcHmT6piYGLZu3Yq/v3+j4gK45557yM7OZuHChWRkZBAbG8umTZtqDbkQQjSNRrH1NuHt1Llz5wgNDSUtLc3m04aEEPbjic9/5YsD5/hjvw4suzsWgLyDy1WNqSm+TjhPSk4JHn3nE39zd7XDsUuLvz7Khz+f4Y5+HVl6d4zd/rzT88v4bH8aro56/jS88zWP9e43v2WCagXk77cQwpZkzZUQQtRTQWkl//n1AtB6B1n8nmVD4a8TziOftdWmKIp1vdX4Xva98N/XzRE0UFJRJfuXCSGESiS5EkKIelp/6BwVVWZ6BLnTL8xL7XBsorO/G3qdhjMXSzl8vkDtcOzOr+cKSC8ox8WgY3g3P7XDuSaDXouXswMAOUVGlaMRQoj2SZIrIYSoh5qDLMJa9SCLKznotUT4Va+1+jrhgsrR2B/LxsGjegTg5KBTOZrr83OTzYSFEEJNklwJIUQ97EvJ5VRWMS4GHZP7tu5BFr9naQ385rd0zGZpDbRQFIVNR9IBmGDnLYEWls2EZaiFEEKoQ5IrIYSoB0vV6g+xIbg7OagcjW2F+7ng7qQno7CcfWdy1Q7HbiRlFnHmYikGvZaRkQFqh1Mv/u5SuRJCCDVJciWEENdxsbiCby9VMKYObBuDLK6k02qtlZmvf5XWQAtLS+BN3fxwc2wdO5dYxrHnlhgxXWOjWiGEEM1DkishhLiOLw6co9KkENPRk94dPdUOp1ncFlPd6rjxcDrGKnlTDpeTq/G9glWOpP7cHfU46rWYFYXcEhlqIYQQLU2SKyGEuAazWeGTfdUtgVMHhakcTfMZ3MUXPzdH8ksr2XkqW+1wVJeSU8LxjCJ0Wg1jolpHSyAAGs0VmwlLciWEEC1NkishhLiGXck5nL1YirujnkkxIWqH02x0Wg239qmu0MjUQKx7Ww3u7IuXi0HlaBrGXyYGCiGEaiS5EkKIa/h4T3XV6o/9OuBiaB3rbhrLkjxuTsykzGhSORp1fXukdWwcXBfrUAuZGCiEEC1OkishhLiKzMJythzLBGDqoLY3yOL3+oV50dHbmVKjiW3HM9UORzUX8sv4NS0fjQbG9gxUO5wGs45jL64ARUbrCyFES5LkSgghruKzX9IwmRVuCPcmMshd7XCanUaj4bZL1av/a8etgZaWwAGdvAlwd1I5mobzdjWg0WgorzRRXFGldjhCCNGuSHIlhBB1MJkV/t0OBln83m2x1cnVj0nZFJRVqhyNOixTAsf1bH0tgQB6nRYf1+q92HJk3ZUQQrQoSa6EEKIOPyRlcaGgHC8XBya0olHcTdUjyIPugW4YTWa+u5RktCc5xRX8cmkj5daaXMEVQy1k3ZUQQrQoSa6EEKIOH++trlrd1b8jTg46laNpWZbWwPa4ofCWxEzMCvTu4Emoj4va4TSanzW5knHsQgjRkiS5EkKI3zmXV8r2pCwApgxsPy2BFpapgT8n55BVVK5yNC1rUyueEngly8RAaQsUQoiWJcmVEEL8zrpf0lAUGNLFl87+bmqH0+I6+boSG+qFWYGNv6WrHU6LKSir5OfkHKB1twTC5cpVXpmRSpNZ5WiEEKL9kORKCCGuUGky8+kvaQBMawfj16/GOjWwHbUGfn88k0qTQrcAN7oGtO6k2sVRj4ujHhS4KNUrIYRoMZJcCSHEFbYmZpJdVIGfmyM3R7e+PY5s5dY+wWg1cCg1n7TcUrXDaRFtpSXQQoZaCCFEy5PkSgghrmAZZHHPDR0x6Nvvr8gADydu7OwLtI/BFqXGKn48kQ20neTKz80AyLorIYRoSe33nYMQQvzOmZwSdp7KQaOBe29of4Msfs/SGvifdpBc/ZiUTXmlmVAfZ6KDPdQOxyYsQy2yi2VioBBCtBS92gEIIYS9sGwaPKK7f6sew20rE3oFs+D/jnA8o4ikjCIig9xrPJ53cLk6gdmQd7/5AGw6eqklsGcQGo1GxYhsx9IWmFNUgaIobeZ1CSGEPZPKlRBCABVVJj7bL4MsruTp4sCI7gEAfP3reZWjaT4VVSa+P1Y9en98G9ow2svFgE6rodJkprCsUu1whBCiXZDkSgghqB5mkFdaSbCnE6Mi/dUOx27cFmtpDUxHURSVo2keP5+6SFFFFQHujvQN9VI7HJvRajX4ylALIYRoUZJcCSEElwdZ3HtDGHqd/Gq0GBMVgLODjtTcUhLS8tUOp1lYpgSO6xmEVtu2Wuf8ZaiFEEK0KHkHIYRo905mFrEvJRedVsM9N4SqHY5dcTHorSPp2+LUwCqTmc2J1cnVhDYyJfBKl8exy1ALIYRoCZJcCSHaPUvVanSPAII8nVSOxv5YpgZ+81s6JnPbag3cdyaXvNJKvF0cGBjho3Y4NudnnRgolSshhGgJklwJIdq1MqOJLw+eA2DajTLIoi43dffH09mB7KIK9p6+qHY4NvXdpZbAm6MD22Q7qN+lylVReSUVlSaVoxFCiLav7f0lEUKIBvjPbxcoKq8i1MeZ4V391A7HLhn0Wib2rm6Z+7+EttMaqCjK5RHsbbAlEMDRQYe7kwMg666EEKIlSHIlhGjXPrnUEjh1YKc2N8zAliZdag389kg6FVVtowKSWVhOZmEFbo56hrbhxNq6mbBMDBRCiGYnyZUQot06cr6AhLR8HHQa7hrQUe1w7NqgCF8C3B0pLK9ix4kctcOxiVNZxQDE9QjAUa9TOZrm4y/j2IUQosXo1Q5ACNF25B1crnYIDZJ0LJNZPgV0D3RHd+IEeYB3v/lqh2WXdFoNt/YJ4f1dKXz96wXrBMFWS1GsyVVbbQm08He/NI69RJIrIYRoblK5EkK0S8YqM8czigDo3cFT5Whahz9c2lB4a2ImJRVVKkfTNDnFRgrKKnHUaxnRvW1vGm0ZanGx2Ii5jU17FEIIeyPJlRCiXUrKKKTSZMbb1UBHb2e1w2kV+nT0pJOvC2WVJrYey1Q7nCY5lVWdWI/o7o+rY9tu4vBwdsBBp8VkVsgvlf2uhBCiOamaXO3YsYNJkyYREhKCRqNhw4YN1scqKyt5+umn6d27N66uroSEhDBjxgwuXKg5qSo8PByNRlPj9sorr9Q45rfffmP48OE4OTkRGhrKa6+9ViuWzz//nB49euDk5ETv3r3ZuHFjs7xmIYQdUBR+O18AXKpaaWSQRX1oNBrrnldft/Kpgaey20dLIFT/3GSohRBCtAxVk6uSkhJiYmJYtWpVrcdKS0s5ePAgCxYs4ODBg6xfv56kpCRuu+22Wse+8MILpKenW2+PPPKI9bHCwkLGjh1Lp06dOHDgAK+//jqLFy/m7bffth7z888/M2XKFO6//34OHTrE5MmTmTx5MkeOHGmeFy6EUFVGYTk5RRXotBqigj3UDqdVsSRXO05mU95K903KKzFysdiIVqNhdI9WvnasniytgbKZsGistLQ0Ro4cSXR0NH369OHzzz9XOyQh7JKqvRATJkxgwoQJdT7m6enJli1baty3cuVKBg4cSGpqKmFhYdb73d3dCQqq+9PHjz/+GKPRyPvvv4/BYKBnz54kJCSwbNkyHnzwQQBWrFjB+PHjefLJJwF48cUX2bJlCytXrmTNmjW2eKlCCDty+Fx11ap7oDtODm13Slxz6BboTo8gd45nFHEqq5herXC9mmWQRaiPC54uDipH0zL8JbkSTaTX61m+fDmxsbFkZGTQv39/Jk6ciKurq9qhCWFXWtWaq4KCAjQaDV5eXjXuf+WVV/D19aVv3768/vrrVFVdXmi9e/dubrrpJgwGg/W+cePGkZSURF5envWYMWPG1DjnuHHj2L1791VjqaiooLCw0HorKiqywSsUQjS3ikoTSZkyyKIpbrs02CIpo3X+3ku+1BLY1d9N5Uhajp9lYmCRrLkSjRMcHExsbCwAQUFB+Pn5kZubq25QQtihVpNclZeX8/TTTzNlyhQ8PC638Tz66KN8+umnbN++nT//+c8sWbKEp556yvp4RkYGgYE12z4sX2dkZFzzGMvjdXn55Zfx9PS03qKjo5v8GoUQze9YeiEms4KfmyPBnk5qh9MqTepTnVydyy9tdVMDi8orySwsBw10Dmg/n7j7uTmi0UCpsarV/czsybXWitelrnXhGo2GuXPnArB48eJaj/Xo0UO1uFetWkV4eDhOTk4MGjSIffv21XncgQMHMJlMhIaG2jxWIVq7VpFcVVZWcvfdd6MoCqtXr67xWHx8PCNHjqRPnz489NBDLF26lLfeeouKiuZtfXj22WcpKCiw3hITE5v1ekIIG1AUDssgiyYL9XGhfydvUOBEK6teWVoCO3g542Jo21MCr6TXafFyuVS9ktbARrvWWvG6/PLLLzXWhFuWO9x1113WY3r27FnjmJ07d17znLt27aKysrLW/YmJiWRm1j3Fsz5xr1u3jvj4eBYtWsTBgweJiYlh3LhxZGVl1TguNzeXGTNm1Fi7LoS4rFHJ1aZNm2r841+1ahWxsbFMnTrV2mpnK5bE6uzZs2zZsqVG1aougwYNoqqqijNnzgDVpevf/7KxfG1Zp3W1Y662jgvA0dERDw8P683d3b2hL00I0cLO55eRW2LEQaelR7D8m20Ky2ALS4tla5Gc1f5aAi0sQy1yZGJgo02YMIGXXnqJ22+/vV7H+/v7ExQUZL198803dOnShREjRliP0ev1NY7x8/O76vnMZjNz585l6tSpmEyXB8okJSURFxfH2rVrGx33smXLeOCBB5g9ezbR0dGsWbMGFxcX3n//fesxFRUVTJ48mWeeeYYhQ4bU63sgRHvTqOTqySefpLCwEIDDhw/z+OOPM3HiRFJSUoiPj7dZcJbE6uTJk2zduhVfX9/rPichIQGtVktAQAAAgwcPZseOHTU+5dmyZQuRkZF4e3tbj9m2bVuN82zZsoXBgwfb7LUIIdRnGWQRGeSOQS+DLJpiYu9gNBrILCynoJXsnVRSUcX5gjIAugS0v+RKhlpc26hRo4iOjq53VaqhjEYj//rXv5gzZw6aK6rmJ0+eJCQkhM6dOzNt2jRSU1Oveg6tVsvGjRs5dOgQM2bMwGw2k5ycTFxcHJMnT66xLKKhsR04cKDG+nOtVsuYMWOs688VRWHWrFnExcUxffr0Rl1HiPagUT0RKSkp1jVGX375JbfeeitLlizh4MGDTJw4sd7nKS4u5tSpUzXOm5CQgI+PD8HBwdx5550cPHiQb775BpPJZF0D5ePjg8FgYPfu3ezdu5dRo0bh7u7O7t27eeyxx7jvvvusidPUqVN5/vnnuf/++3n66ac5cuQIK1as4I033rBe9y9/+QsjRoxg6dKl3HLLLXz66afs379fSt5CtCFlxipOXhpkIIMsms7f3ZFQHxdSL5aSlFHEwM7X//BLbaezi0GBQA8n3J3ax5TAK8leV9e2fft2Onbs2Gzn37BhA/n5+cyaNct636BBg/jwww+JjIwkPT2d559/nuHDh3PkyJGrdsSEhITw/fffM3z4cKZOnWodyvX7ZRMNkZOTg8lkqnP9+fHjx4HqdsR169bRp08f65qtjz76iN69ezf6ukK0RY1KrgwGA6WlpQBs3bqVGTNmANVJj6WiVR/79+9n1KhR1q8tVa+ZM2eyePFivv76awDrdBqL7du3M3LkSBwdHfn0009ZvHgxFRUVRERE8Nhjj9Wonnl6erJ582bmzp1L//798fPzY+HChdYx7ABDhgzhk08+4bnnnuOvf/0r3bp1Y8OGDfTq1ath3xghhN1KvFCI2awQ6OFEgIcMsrCFyED36uQqs4iBET52v4bNsnFw13ZYtYLLbYF5pZWUV5pkG4IW9t577zFhwgRCQkKs9125HU2fPn0YNGgQnTp14rPPPuP++++/6rnCwsL46KOPGDFiBJ07d+a9996rUQ1rDsOGDcNsNjfrNYRoCxqVXA0bNoz4+HiGDh3Kvn37WLduHQAnTpxo0Kc+I0eORFGUqz5+rccA+vXrx549e657nT59+vDTTz9d85i77rqrxgJTIUTboSgKhy9cMchC2ESXADe+P55FbomRnGIjfpcqI/aootLEudzqlsD2uN4KwM1Rh5ODjvJKEyczi+ndUf4ttJSzZ8+ydetW1q9ff83jvLy86N69e42unrpkZmby4IMPMmnSJH755Rcee+wx3nrrrUbH5+fnh06na/D6cyFEbY1ac7Vy5Ur0ej1ffPEFq1evpkOHDgB8++23jB8/3qYBCiFEU6XlllJQWolBr6V7kAyysBVHvY5w3+px5va+59Xp7BLMioKvmyNerobrP6Et0misrYHH0uvfZSKa7oMPPiAgIIBbbrnlmscVFxeTnJxMcHDwVY/Jyclh9OjRREVFsX79erZt28a6det44oknGh2fwWCgf//+Ndafm81mtm3bJuvPhWigRlWuwsLC+Oabb2rdf+U6JiGEsBeWQRZRQR446FrFDhStRmSQO8nZxSRlFjG0q6/dtgaeaocbB9fF382RtNxSEiW5apRrrRUPCwtj5cqVfPXVV7WSlA8++ICZM2ei19d82/XEE08wadIkOnXqxIULF1i0aBE6nY4pU6bUeX2z2cyECRPo1KkT69atQ6/XEx0dzZYtW4iLi6NDhw489thjDY4bqpdmzJw5kwEDBjBw4ECWL19OSUkJs2fPbtL3TIj2ptGbfCQnJ/PBBx+QnJzMihUrCAgI4NtvvyUsLIyePXvaMkYhhGi0kooqknNKAKQNqhlE+LnioNNSVF5JekE5wV7OaodUS2WVmbMXq/8faK/rrSws664kuWqca60V//DDD8nJySE5ObnGc7Zu3Upqaipz5sypdb5z584xZcoULl68iL+/P8OGDWPPnj34+/vXeX2tVsuSJUsYPnw4BsPlCmxMTAxbt2696vOuFzfAPffcQ3Z2NgsXLiQjI4PY2Fg2bdpUa8iFEOLaGpVc/fjjj0yYMIGhQ4eyY8cO/va3vxEQEMCvv/7Ke++9xxdffGHrOIUQolGOni9AURRCvJzxdbPfNUGtlV6npYu/G8czCknKKLLL5OrMxRJMZgVPFwf83NppS+AlV7YFKorS7EMQ2prrrRVfvHgxixcvrnHf2LFjr/qcTz/9tMEx3HzzzXXe37dv36s+53pxW8ybN4958+Y1OCYhxGWN6o955plneOmll9iyZUuNT07i4uLqNWBCCCFagqIoHJFBFs0u8tI6tpNZRZjN138D19JOXblxcDtPJrxdDWi1GorKqzifX6Z2OEII0eY0Krk6fPhwnbt8BwQEkJOT0+SghBDCFs7klFBUXoWTg67dt4M1pzAfF5wcdJQaTZzLK1U7nBqqTGZScqQl0EKn1eDjUv2h6LF0+x5CItqesrIy61Y+UD1Fcfny5WzevFnFqISwrUYlV15eXqSnp9e6/9ChQ9bJgUIIobbD56urVtHBHuhlkEWz0Wo1dLuUuNjb1MC03FIqTWbcHPUEyv5mwOXWwMQLsu5KtKw//OEP/POf/wQgPz+fQYMGsXTpUv7whz80aRNkIexJo95t3HvvvTz99NNkZGSg0Wgwm83s2rWLJ554wrqhsBBCqKmwrJKUS0MMpCWw+VlaA09lF1Nlsp+NRq0tgQFusr7oEn83Gccu1HHw4EGGDx8OwBdffEFgYCBnz57ln//8J2+++abK0QlhG41KrpYsWUKPHj0IDQ2luLiY6OhobrrpJoYMGcJzzz1n6xiFEKLBjl4oAAVCfVza775GLSjEyxk3Jz3GKjNnL9pHa6DZrHD6Uktgl3Y+gv1Kls2ej2VIciVaVmlpKe7u1R/EbN68mT/+8Y9otVpuvPFGzp49q3J0QthGo5Irg8HAO++8w+nTp/nmm2/417/+xfHjx/noo4/Q6XS2jlEIIRrEbFY4cqnlSapWLUOj0RAZWP2myV5aA8/nl1FeacLZoKODt/1NMVSLZRz72YulFFdUqRyNaE+6du3Khg0bSEtL47vvvmPs2LEAZGVl4eHhoXJ0QthGo5KrF154gdLSUkJDQ5k4cSJ333033bp1o6ysjBdeeMHWMQohRIOczi6mtKIKF4OezlKxaDGW5Op0TjHGKvVbA09lVSd5XfykJfBKzgYdQZfWnx2X1kDRghYuXMgTTzxBeHg4gwYNYvDgwUB1Fetao+SFaE0alVw9//zzFBcX17q/tLSU559/vslBCSFEU1gGWfQM8UCnlTfVLcXf3RFvFwMms8Lp7Np/I1qSoiicyr7UEihTAmuJCq5OhGXdlWhJd955J6mpqezfv59NmzZZ7x89ejTLly9XLzAhbKhRydXVNh789ddf8fHxaXJQQgjRWAWlRlJzS0EDvaQlsGVpNHS3k9bAjIJySiuqMOi1hPq4qBqLPYoKrm7BSpRx7KIFzZkzB1dXV/r27YtWe/ktaM+ePXn11VdVjEwI22lQcuXt7Y2Pjw8ajYbu3bvj4+NjvXl6enLzzTdz9913N1esQghxXZaqVbiPKx7ODipH0/5Ypgam5pZSZjSpFodlSmBnP1epXtbBklxJ5Uq0pLVr11JWVnvz6rKyMuuIdiFaO31DDl6+fDmKojBnzhyef/55PD0vfypsMBgIDw+39s8KIURLM5nNHLUMsugoVSs1eLsa8Hd3JLuoglNZRfTu6NXyQSgKp7Ivj2AXtUWHVCdXxzMKMZkVSUBFsyosLERRFBRFoaioCCeny3vOmUwmNm7cSEBAgIoRCmE7DUquZs6cCUBERARDhgzBwUE+FRZC2I9TWcWUV5pwc9IT7uuqdjjtVo8gd7KLKkjKVCe5yi6qoLCsEr1OQyf5/6BO4b6uODloKa80c+ZiiYyqF83Ky8sLjUZj7Xz6PY1GI2v2RZvRoOTKYsSIEZhMJr788kuOHTsGVPfL3nbbbTKKXdSQd3C52iE0mXe/+WqHIOrJ0hLYK8QTrXwSr5puge78dDKH83llFJVX4u7Ush/EWVoCw31c0esatbS4zdNpNUQGefBrWj7H0gsluRLNavv27SiKQlxcHF9++WWN9fkGg4FOnToREhKiYoRC2E6jkqtTp04xceJEzp8/T2RkJAAvv/wyoaGh/Pe//6VLly42DVIIIa4nt7iC83llaDQaeoZIS6Ca3J0c6ODlzPn8Mk5mFtGvU8sOOrK2BAZKwnAt0cHu1uTq1j7yxlY0nxEjRgCQkpJCWFiYbI0g2rRGfaT36KOP0qVLF9LS0jh48CAHDx4kNTWViIgIHn30UVvHKIQQ12WpWnX2c8XNqVGfGwkb6n5psEVSZstOo8srqSC3xIhWqyHcT1oCryXaMjHwggy1EC3j2LFj7Nq1y/r1qlWriI2NZerUqeTl5akYmRC206jk6scff+S1116rUdb19fXllVde4ccff7RZcEIIUR9VJjOJ6TLIwp50C6jeuDersIL8EmOLXdfSEhjm7YKjXtrUr+XyxEAZxy5axpNPPklhYfXv6sOHDxMfH8/EiRNJSUkhPj5e5eiEsI1GJVeOjo4UFdX+ZVxcXIzBYGhyUEII0RAnMoswVpnxcHYgTPY0sgvOBj2dLv0sWrJ6JVMC66/HpeQqo7CcvBZMgEX7lZKSQnR0NABffvklkyZNYsmSJaxatYpvv/1W5eiEsI1GJVe33norDz74IHv37rWO1tyzZw8PPfQQt912m61jFEKIa7K0BPbu4Cm9/HYk8srWQEVp9usVllWSVViBRgOd/aUl8HrcHPXWDyNkvyvREgwGA6WlpQBs3bqVsWPHAuDj42OtaAnR2jUquXrzzTfp0qULgwcPxsnJCScnJ4YOHUrXrl1ZsWKFrWMUQoiryi4sJ6OgHK1WY927R9iHzv5u6LQa8kqMZBdVNPv1ki+1BHbwcsbZIOvu6sO67kqSK9EChg0bRnx8PC+++CL79u3jlltuAeDEiRN07NhR5eiEsI1G/fXx8vLi//7v/zh58iTHjx8HICoqiq5du9o0OCGEuB5L1aqrvxsu8obarhj0Wjr7uXIyq5ikzCL8PZyu/6QmOGltCXRv1uu0JVHBHmw6miHJlWgRK1eu5OGHH+aLL75g9erVdOjQAYBvv/2W8ePHqxydELbRqHciO3fuZNiwYXTr1o1u3brZOiYhhKgXY5WZ4xnV63l6d5BBFvYoMsjdmlwN7erXbG2bJRVVpOeXAcieTQ0QFVydiMpQC9ESwsLC+Oabb2rd/8Ybb6gQjRDNo1HJVVxcHB06dGDKlCncd9991sWJQgjRkpIyCqk0mfF2NdDR21ntcEQdOvm6YtBrKS6vIj2/nJBm+jklX6paBXk6ySj+BrBMDDyVVT0UxqCXTZdF8zKZTGzYsIFjx44B0LNnT2677TZ0OpnuKdqGRv0WvXDhAo8//jg//vgjvXr1IjY2ltdff51z587ZOj4hhKibovDbFYMskEEWdkmv09L1UiUpKbP5Ws8sI9i7StWqQTp6O+PupKfSpFi/h0I0l1OnThEVFcWMGTNYv34969ev57777qNnz54kJyerHZ4QNtGo5MrPz4958+axa9cukpOTueuuu1i7di3h4eHExcXZOkYhhKglo7CcnKIKdFqN9dN3YZ8sUwNPZBZjNtt+amC50cS5vOqWQBnB3jAajeaK/a5k3ZVoXo8++ihdunQhLS2NgwcPcvDgQVJTU4mIiODRRx9VOzwhbKLJvRMRERE888wzxMTEsGDBAtlEWAjRIg6fq65adQ90x8lB2knsWUdvF1wMOkqNJlJzSwn3s+2Y9NM5xSiKgp+7I54ustdiQ0UHe7AvJVeSK9HsfvzxR/bs2YOPj4/1Pl9fX1555RWGDh2qYmRC2E6Tmqt37drFww8/THBwMFOnTqVXr17897//tVVsQghRp4pKk3VjWhlkYf+0Wg3dLk3wO5Fh+8EJ0hLYNNahFhmSXInm5ejoSFFR7d8BxcXFGAzywYhoGxqVXD377LNEREQQFxdHamoqK1asICMjg48++khGaQohmt2x9EJMZgU/N0eCPZt3vLewDUtr4KnsYqpMZpud11hlJjW3elNSaQlsnMttgUUoLbDZs2i/br31Vh588EH27t2LoigoisKePXt46KGHuO2229QOTwibaFRb4I4dO3jyySe5++678fPzs3VMQghxdYpi3dtKBlm0HkGeTrg7OVBUXsmZnBK6BtpmL6ozOSWYzAreLgZ8XeWT78boHuiOTqsht8RIZmEFQfKBhWgmb775JjNnzmTw4ME4ODgAUFVVxW233caKFStUjk4I22hUcrVr1y5bxyGEEPVyPr+M3BIjDjotPYJls9jWQqPREBnozv6zuSRlFNksubK0BHYJcJNEu5GcHHTWzZ6PpRdKciWajZeXF//3f//HyZMnOXbsWPVAlagounbtqnZoQthMowdaJCUl8dZbb1n3KYiKiuKRRx4hMjLSZsEJIcTvWQZZRAa5Y9DLIIvWJDKoOrlKuVhCRZUJxyb+/KpMZs5cLAFkvVVTRQV7cDKrmMT0Qkb1CFA7HNHGdevWzZpQNdfG4kKopVFrrr788kt69erFgQMHiImJISYmhoMHD9KrVy++/PJLW8cohBAAlFeaOJdfPXJbBlm0Pn5uBnxcDZjMCsk22FMp9WIplSYz7k56Aj0cbRBh+yXj2EVLee+99+jVqxdOTk44OTnRq1cv3n33XbXDEsJmGlW5euqpp3j22Wd54YUXaty/aNEinnrqKe644w6bBCeEEFBdodBoNFSZFGYOCSezoJwAD2ldanUutQbuPn2RpMwiokOaliCfyr7UEugvLYFNFR1SnVwlSnIlmtHChQtZtmwZjzzyCIMHDwZg9+7dPPbYY6SmptZ6XylEa9So5Co9PZ0ZM2bUuv++++7j9ddfb3JQQghhYTKZ2X8mj4RzeVRUmnF00BLb0YtgTyd0uibtJiFUEBlUnVyl5ZZRaqzCxdC47nSzWeH0peTKMuZdNJ5lHPuZnBLKjCacDdJyK2xv9erVvPPOO0yZMsV632233UafPn145JFHJLkSbUKj3pmMHDmSn376qdb9O3fuZPjw4U0OSgghoLpi9cuZPPamXKSisnp8d0Wlmb0pufxyJo9KG470Fi3D08VAoIcTiqJwKrPxrYHn8kqpqDLjYtAR7CVVzKYKcHfCz82AWcG6h5wQtlZZWcmAAQNq3d+/f3+qqqpUiEgI22vUR4a33XYbTz/9NAcOHODGG28EYM+ePXz++ec8//zzfP311zWOFUKIxtBoNCScy6vzsYRzedwQ4dPCEQlbiAx0J7OwnOOZRfQJ9WrUOSxTAjv7u8mCeBuJCvbgp5M5HEsvJLaRPxchrmX69OmsXr2aZcuW1bj/7bffZtq0aSpFJYRtNSq5evjhhwH4xz/+wT/+8Y86H4PqN0Ymk6kJ4Qkh2jNjlclasfq9ikozxioTzo1sKxPq6R7kzo5T2aTnl1FYVomHs0ODnq8oCsnZ1VMCu8nGwTYTfSm5Srwg665E83nvvffYvHmz9cP5vXv3kpqayowZM4iPj7ce9/sETIjWolHvSsxmacURQjQ/g16Ho4O2zgTL0UEro9hbKVdHPR29nDmXV8aJzCIGhDesApmeX06psQpHvZYO3i7NFGX7IxMDRXM7cuQI/fr1AyA5ORkAPz8//Pz8OHLkiPU4qUaL1kw+8hVC2C1FUYjt6MXelNxaj8V29MasKOiQP8KtUWSgB+fyykhqRHJlbQn0c0OnlZ+/rViSq+MZRZjNClr53rYbaWlpTJ8+naysLPR6PQsWLOCuu+6y+XW2b99er+POnTuH2WxGq5WhRaL1aVRydb1pLgsXLmxUMEIIcaWCskpiQr1QgF/P5V8xLdCbG8K9ZVpgK9Y1wI3tSVnkFFWQV1KBt2s996lSFOsI9q7SEmhTnf1dMei0FFdUcS6vjDBfqQq2F3q9nuXLlxMbG0tGRgb9+/dn4sSJuLq6qhJPdHQ0CQkJdO7cWZXrC9EUjUquvvrqqxpfV1ZWkpKSgl6vp0uXLpJcCSGaTFEUtiRmYjSZGRsdyMAIX4xVJgx6XXXFShKrVs3JoKOTrwspOSUkZRRxY5f6JVdZRRUUlVfioNPKm38bc9Bp6RboxtELhSSmF8j3tx0JDg4mODgYgKCgIPz8/MjNzVUtuVIURZXrCmELjXp3cujQoRq3I0eOkJ6ezujRo3nsscdsHaMQoh06fL6AzMJySiqqcHdyQKfV4GzQo9NqcJDEqk2IDKreW+l4ZhHU882UpSUw3NcVvfx/YHPRwZbNhGUce1127NjBpEmTCAkJQaPRsGHDhmseHx4ejkajqXWbO3dui8e0atUqwsPDcXJyYtCgQezbt6/O4w4cOIDJZCI0NNRmMQrRntjsL5OHhwfPP/88CxYssNUphbiqKpMZk1mhzFiFyazIfkdtTGlFFbtO5QAwpIsfro6yPLQt6uznhl6noaC0kszCius/QVGsyZW0BDYPGWpxbSUlJcTExLBq1ap6Hf/LL7+Qnp5uvW3ZsgXgquuZdu3aRWVlZa37ExMTyczMbHRM69atIz4+nkWLFnHw4EFiYmIYN24cWVlZNY7Lzc1lxowZvP322/V6fUKI2mz6sV9BQQEFBQW2PKUQtZhMZvafyeOdn5J5e8dp3vkpmQNn8jBJgtVm7DyVg7HKjL+7I306eqodjmgmDnotnf2qk6T6bFybW2Ikr9SITqsh3E9a1ppDe06uRo0aRXR09DWTlAkTJvDSSy9x++231+uc/v7+BAUFWW/ffPMNXbp0YcSIEbWONZvNzJ07l6lTp9bYxiYpKYm4uDjWrl3b6JiWLVvGAw88wOzZs4mOjmbNmjW4uLjw/vvvW4+pqKhg8uTJPPPMMwwZMqRer08IUVujPg5+8803a3ytKArp6el89NFHTJgwwSaBCVGXqkuJ1d6Ui9b7KirN1q/7h3tLy1grdz6vtPqNnQbiegTISN42LjLQnROZRZzILGJ4N79r/rwtVaswHxcZw99MLG2B5/LKKCirxLOBe5C1Ztu3b6djx47Ndn6j0ci//vUv4uPj6/z/XKvVsnHjRm666SZmzJjBRx99REpKCnFxcUyePJmnnnqq0dc9cOAAzz77bI1rjRkzht27dwPV7+NmzZpFXFwc06dPb9wLtCH5vS9as0YlV2+88UaNr7VaLf7+/sycObPGP14hbE2j0ZBwLq/OxxLO5XFDRMNGOgv7YjYrfH+8uk2ld4gnQZ7OKkckmlsnPxcc9VpKKqo4n1dGR5+rV6ROypTAZufp4kAHL2fO55dxPL2QQZ191Q6pzdiwYQP5+fnMmjXrqseEhITw/fffM3z4cKZOncru3bsZM2YMq1evbvR1c3JyMJlMBAYG1rg/MDCQ48ePA9XtiOvWraNPnz7WNVsfffQRvXv3bvR1m0IGWojWrFHJVUpKSr2OO3fuHCEhIbJPgbAZY5Wpzg1lobqCZawy4WyQ9Tmt1aHUPHJLjDgbdAzp6qd2OKIF6LRauga4c/RCAUmZRVdNrgpKjeQUVaDRaKythKJ5RAW7cz6/jGOSXNnUe++9x4QJEwgJCbnmcWFhYXz00UeMGDGCzp0789577zV7JWfYsGGYzfbTWp+YmHjd75MQ9qpZs57o6GjOnDnTnJcQ7YxBr8PRoe7/bR0dtNIq1IoVlVey59JmwcO6+uHkID/L9sIyNfBkVjEmc92fWFtaAjt6O+NkkP83mtPldVcyMdBWzp49y9atW/nTn/503WMzMzN58MEHmTRpEqWlpU2ewuzn54dOp6s1ECMzM5OgoKAmnbuhysvLef3115k4cSIDBgygX79+NW4WoaGh6HTy71y0Ts36Eb+UdYWtKYpCbEfvGmuuLGI7elfvf4T0ardGO5KyqTKZCfFytr65E+1DR29nXBz1lFZUkXqxhAj/2pUp68bBdTwmbCvKOo69/Q21aC4ffPABAQEB3HLLLdc8Licnh9GjRxMVFcXnn3/OiRMnGDlyJI6Ojvz9739v1LUNBgP9+/dn27ZtTJ48GagenrFt2zbmzZvXqHM21v3338/mzZu58847GThwoKytEm2S9E+JVkWv09K/kzcKCr+ey6ei0oyjg5aYjl7Ehnmh18ov6tboTE4Jp7KL0Wg0MsSiHdJoNHQPcCMhLZ+kzKJayVVxeRUZBeWggS6y3qrZWYZaJGUWUWUyy35iVyguLubUqVPWr1NSUkhISMDHx4ewsDBWrlzJV199xbZt26zHmM1mPvjgA2bOnIlef/W3XWazmQkTJtCpUyfWrVuHXq8nOjqaLVu2EBcXR4cOHeqsYl0vJoD4+HhmzpzJgAEDGDhwIMuXL6ekpITZs2fb4ttSb9988w0bN25k6NChLXpdIVqSJFeidVEUvj+eSbcAd/40rDOVJjMGvY6zF0v4bH8aMR29iAn1UjtK0QBVJjM/JFUPsegb6oWvm6PKEQk1RAa5k5CWz+nsklpv6JMvVa2CPZ1lz7MWEObjgqtBR4nRREpOCd0C3dUOyW7s37+fUaNGWb+Oj48HYObMmXz44Yfk5OSQnJxc4zlbt24lNTWVOXPmXPPcWq2WJUuWMHz4cAwGg/X+mJgYtm7dir+/f6NiArjnnnvIzs5m4cKFZGRkEBsby6ZNm2oNuWhuHTp0wN1d/n8SbZt8HCValbS8Mo5nFLHxSDrGKjPOBj06rYbi8irySozsPJVNfolR7TBFA+w/k0dBWSVujnpZPN+OBXk44eHsQKXJzOnskhqPWTcOlpbAFqHVaqzr4KQ1sKaRI0eiKEqtmyWJWbx4ca215mPHjkVRFLp3737d89988804OTnVur9v375XHRN/vZgs5s2bx9mzZ6moqGDv3r0MGjSoXq/ZlpYuXcrTTz/N2bNnW/zaQrSUZk2upLVH2JSisOd09Vqr3h08cbniE+zeHT0J9XGhyqTwXWIG5qssihf2Jb/EyP6z1UMsRnT3x6CXz3vaLY2GyEsVkqSMy4MUyowmzueXAjKCvSXJUAvRHAYMGEB5eTmdO3fG3d0dHx+fGjch2gIZaCFajbTcMi7kl6HTahgQXvOXsEaj4eboQP615ywZBeUcOCt7Xtk9RWF7UhYms0InX1d54yyIDHLnlzO5nMktoaLShKODjtPZxSgK+Ls74tGONrRVW3SIDLUQtjdlyhTOnz/PkiVLCAwMlA/hRZvUrMmV7FMgbEZR2JNyuWpV17oLdycHRnYPYHNiBntSLhLh54qfu6zfsVcns4pJzS1Fp9UwKtIf5I9su+fr5oivmyMXiys4lVVMzw6e0hKoksuVK0muhO38/PPP7N69m5iYGLVDEaLZ1Du5+uMf/1jvk65fvx6o3qdACFu4VtXqSlHB7pzKLuZ0djHfJWZw7w1h6GSCoN0xVpn48UQ2ADeE++DpYrjOM0R7ERnozs/FFSRlFtE10I20vOqWwG6Bkly1pB5B7mg0kF1UQU5xBX4yaEbYQI8ePSgrK1M7DCGaVb0XOHh6etb7JoRNXVG16tPB69rTwjQaRvcIwNmgI6eogr2na++HJdS353QuJRVVeLo40L+Tt9rhCDtiGaRwsbiC01nFGPRavF0NeLvKm/uW5GLQE+7rCkj1StjOK6+8wuOPP84PP/zAxYsXKSwsrHEToi2od+Xqgw8+sPnFd+zYweuvv86BAwdIT0/nq6++sm5wV1lZyXPPPcfGjRs5ffo0np6ejBkzhldeeaVGq2Fubi6PPPII//nPf9Bqtdxxxx2sWLECN7fLn3L+9ttvzJ07l19++QV/f38eeeQRnnrqqRqxfP755yxYsIAzZ87QrVs3Xn31VSZOnGjz1ywaLjW31Fq16h9+/TfiLo564iID+O/hdPafzaWzvytBns4tEKmoj5yiChLS8gEY1T1A9tARNXg4O3BH/44EejhRZjQxe2gEBaWVaofVLkUFu5OSU0LihUKGd6t7DLgQDTF+/HgARo8eXeN+RVHQaDSYTCY1whLCplTdMKSkpISYmBjmzJlTq+2wtLSUgwcPsmDBAmJiYsjLy+Mvf/kLt912G/v377ceN23aNNLT09myZQuVlZXMnj2bBx98kE8++QSAwsJCxo4dy5gxY1izZg2HDx9mzpw5eHl58eCDDwLVPcBTpkzh5Zdf5tZbb+WTTz5h8uTJHDx4kF69erXcN0TUpijsSameJnfdqtUVuga60yO7mOMZRWw+msnUQWHyJt4OKIrC90lZKIpCtwA3Ovm5qh2SsDMmk5lzuaV889sF6ybhsR298XZxQCf/hltUdLAHGw9nSOVK2Mz27dvVDkGIZtfo5OqLL77gs88+IzU1FaOx5r5CBw8erNc5JkyYwIQJE+p8zNPTky1bttS4b+XKlQwcOJDU1FTCwsI4duwYmzZt4pdffmHAgAEAvPXWW0ycOJG///3vhISE8PHHH2M0Gnn//fcxGAz07NmThIQEli1bZk2uVqxYwfjx43nyyScBePHFF9myZQsrV65kzZo1dcZXUVFBRUWF9euiIhlX2xxSc0tJb0DV6kojIwNIyysjr9TIrlM5jIgMaKYoRX0du1BIen4ZDjotN3WXT8JFTVUmM/vP5LH30gcqABWVZvZeagvuH+6NgyRYLUbGsQtbGzFihNohCNHsGvVX6s0332T27NkEBgZy6NAhBg4ciK+vL6dPn75qsmQLBQUFaDQavLy8ANi9ezdeXl7WxApgzJgxaLVa9u7daz3mpptuqrHb+bhx40hKSiIvL896zJgxY2pca9y4cezevfuqsbz88ss11plFR0fb6mUKi0ZWrSwcHXTcHFW9+3xCWj7nckttHqKov3KjiZ9O5QBwY2df3JxkrLaoSaPRkHAur87HEs7loZWJki3KklwlZxdTXintWqLpduzYcc2bEG1BoypX//jHP3j77beZMmUKH374IU899RSdO3dm4cKF5ObmXv8EjVBeXs7TTz/NlClT8PCo/oWfkZFBQEDNaoRer8fHx4eMjAzrMRERETWOCQwMtD7m7e1NRkaG9b4rj7Gcoy7PPvss8fHx1q/Pnz8vCZaNNaVqZdHJz5XeHTw5fL6AzYmZTLsxDEe9zsaRivrYlZxDeaUJXzdHYkO91A5H2CFjlYmKSnOdj1VUmjFWmXA2qNrN3q4Eezrh6exAQVklp7KK6dVBBlaJphk5cmSt+67c60rWXIm2oFGVq9TUVIYMGQKAs7OztSVu+vTp/Pvf/7ZddJdUVlZy9913oygKq1evtvn5G8PR0REPDw/rzd3dXe2Q2hZFYc+lSX99Oja8anWl4d388XR2oKi8kp8ujf8WLSujoIwj5wsAiOsRgFbG44s6GPQ6HB3q/rPk6KDFIB+MtCiNRkN0sGwmLGwnLy+vxi0rK4tNmzZxww03sHnzZrXDE8ImGpVcBQUFWStUYWFh7NmzB4CUlBQURbFddFxOrM6ePcuWLVusVStLHFlZWTWOr6qqIjc3l6CgIOsxmZmZNY6xfH29YyyPi5aXmltKekF5ddWqiaO6HfRaxkYHgQaOXigkJbvYRlGK+jCbFb4/Xv3vNDrYgxAvmdwo6qYoCrEd6/73HtvRG7ON/76I65PNhIUt/X7rHj8/P26++WZeffXVWlOchWitGpVcxcXF8fXXXwMwe/ZsHnvsMW6++Wbuuecebr/9dpsFZ0msTp48ydatW/H19a3x+ODBg8nPz+fAgQPW+77//nvMZjODBg2yHrNjxw4qKy+P8t2yZQuRkZF4e3tbj9m2bVuNc2/ZsoXBgwfb7LWIBrBh1coixNuZfqHVP++tx7IoM0rrQUv57Vw+2UUVODroGNbNT+1whB3T67TcEO7NoAhfawXL0UHLoAhfbpBhFqqICq7uypDkSjSnwMBAkpKS1A5DCJto1LvWt99+G7O5ui9+7ty5+Pr68vPPP3Pbbbfx5z//ud7nKS4u5tSpU9avU1JSSEhIwMfHh+DgYO68804OHjzIN998g8lksq6B8vHxwWAwEBUVxfjx43nggQdYs2YNlZWVzJs3j3vvvde6F9bUqVN5/vnnuf/++3n66ac5cuQIK1as4I033rBe9y9/+QsjRoxg6dKl3HLLLXz66afs37+ft99+uzHfHtFElqqVXtf0qtWVBnfx5czFEnJLjGw/nsXE3kEgC+SbVUlFFbsvJcpDu/jKehlxXTqdlv7h3twQ4YOxyoRBr8OsKDKGXSWWylXihULrXkRCNNZvv/1W42tFUUhPT+eVV14hNjZWnaCEsLFGvdM5d+4coaGh1q/vvfde7r33XhRFIS0tjbCwsHqdZ//+/YwaNcr6tWVAxMyZM1m8eLG1Ovb7f3Dbt2+3Lor8+OOPmTdvHqNHj7ZuIvzmm29aj/X09GTz5s3MnTuX/v374+fnx8KFC61j2AGGDBnCJ598wnPPPcdf//pXunXrxoYNG2SPKzUoivXNeGMmBF6LXqdlXM8g1v2SxsmsIk5kutE9SNbKNaefTmRjrDIT6Okki+FFvVkqVJZkXIe8oVdLt0A39FoNheVVXCgop4O09YomiI2NRaPR1FpCcuONN/L++++rFJUQttWod64RERGkp6fXmtSXm5tLREREvae9jBw58pprtOqzfsvHx8e6YfDV9OnTh59++umax9x1113cdddd172eaF5nc0vJuFS16mfDqpVFgIcTAyN82HP6It8nZdHB29mmCZy4LC23lKTMIjQaiIsMkE+8hWiFHPU6uga4cTyjiGMXCiW5Ek2SkpJS42utVou/vz9OTk4qRSSE7TXqXeXVWgOKi4vlH4hovCvXWtm4anWlG8J9OJ1TQlZhOVsTM/lDbIi0B9qYyWxme1L1EIs+HbwI8JDfC0K0VlHBHtXJVXohY6IDr/8EIa6iU6dObNu2jW3btpGVlWVdYmIh1SvRFjTo3aulbU+j0bBgwQJcXFysj5lMJvbu3Ss9s6LRrqxa2XKt1e9ptRrGRQfyyb5Uzlws4ciFQmlZs7GDZ/PJKzHiYtAzuIvv9Z8ghLBbUcHufHVIxrGLpnv++ed54YUXGDBgAMHBwdLRINqkBiVXhw4dAqorV4cPH8ZgMFgfMxgMxMTE8MQTT9g2QtE+/K5q5dLMrXo+bo4M6eLHTyez2XEimzAfFzycHZr1mu1FYVkle1Oqf5Y3dfPD0UH2JhKiNYsOrv7wSSYGiqZas2YNH374IdOnT1c7FCGaTYPewW7fvh2oHr++YsWKGntOCdEUZy+2TNXqSn3DvDidXcz5/DI2J2ZwR7+O8imaDfyYlIXJrNDR25lIGRgiRKtnGcd+NreUkooqWacqGs1oNDJkyBC1wxCiWTVqtu0HH3xgTazOnTvHuXPnbBqUaGcUhT0pLVe1stBoNNwcHYSDTsv5vDISUvNb5Lpt2ensYk7nlKDVaIjrESBr2YRoA3zdHAlwd0RR4HhGkdrhiFbsT3/603WHkAnR2jXqXazZbOall15i6dKlFBcXA+Du7s7jjz/O//7v/6LVyn4kov7UqFpZeLo4MLybH98fz2JXcg7hfi54uzq2aAxtRamxih+SsgHo18lbvo9CtCFRwR5kFWWTmF7Y4r+nRdtRXl7O22+/zdatW+nTpw8ODjXb8ZctW6ZSZELYTqOSq//93//lvffe45VXXmHo0KEA7Ny5k8WLF1NeXs7f/vY3mwYp2jBFYbcKVasr9e7gSXJ2CWcvlvDd0UzuHhCKVisVl4Za+f0pSssrcXdyYGCEj9rhCCFsKDrEgx9PZMu6K9Ekv/32m3Xw2ZEjR2o8Jm35oq1o1DvZtWvX8u6773LbbbdZ7+vTpw8dOnTg4YcfluRK1NuZi6VkWqpW4Sp9GqrRMCYqkH/tOUNmYTn7z+QysLNMuGuIU1lFvPPTaaZ5wsju/tZNYIUQbUNUcPVSAEmuRFNY1u4L0ZY16h1Qbm4uPXr0qHV/jx49yM3NbXJQop24Yq1VTEcvXAzqLZJ2c9Izqkf1pth7U3LJLixXLZbWRlEUFmw4SqVJIcLPlc7+rmqHJISwsehLQy2SMoowmxWVoxFCCPvVqOQqJiaGlStX1rp/5cqVxMTENDko0T5cWbXqZwc9/JGB7nQNcMOsKHyXmEGVyXz9Jwn+L+ECu09fxFGvZWSkDLEQoi0K93XFUa+l1GjibG6p2uEIIYTdalSp4LXXXuOWW25h69atDB48GIDdu3eTlpbGxo0bbRqgaKPsqGpldWnC3YX8Mi4WG9lz+iLDuvmrHZVdKyir5KX/HgPgkbiueDgnqRyREKI56HVaIoPc+e1cAYkXConwkwq1EELUpVGVq4iICE6cOMHtt99Ofn4++fn5/PGPfyQpKYlOnTrZOkbRBl2uWmntompl4WzQMzoqEIADqXmk55epHJF9W7Y5iZziCjr7ufLATZ3VDkcI0YyiZd2VEEJcV6PKBREREaSnp9caXHHx4kVCQ0MxmUw2CU60UYrCntOWqpWnfVStrtDZ343oYA8S0wv5LjGDTkNl08y6HD5XwEd7zgLwwh964ajXIc1CQrRdMtRCCCGur1GVK0WpezFrcXExTk5OTQpItH1nLpaSWWh/Vasr3RTpj7uTnoLSSl7+9pja4dgdk1nhuQ2HMSswKSaEYd381A5JCNHMJLkSQojra9DH8fHx8UD1XgQLFy7ExcXF+pjJZGLv3r3W/QuEqJOdV60sHPU6bo4OYv3Bc/xrTypjo4O4qbusv7L49JdUfj1XgJujnuduiVI7HCFEC+hxaWLghYJy8kuNeLkYVI5ICCHsT4Pe2R46dAiorlwdPnwYg+HyL1aDwUBMTAxPPPGEbSMUbcqZiyV2X7WyCPVxIaajF+TCU1/8xneP3YSns8N1n9fW5RRX8Nqm6sEVj4/tTqCHVKuFaA88nBwI9XEmLbeMxPRChnSRirUQQvxeg5Iry+Zvs2fPZsWKFXh4eDRLUKKNUhT2nK7eB82eq1ZXGtbNj4gLrqTklPD810dZdk+s2iGp7pVvj1NQVkl0sAfTb5QBNkK0J1FBHqTllnEsvUiSKyGEqEOj1lx98MEHkliJBruyatXfzqtWFnqdlr/fFYNWA+sPnWfTkQy1Q1LVvpRcvjhwDoCXbu+FXteoXyFCiFZK1l0JIcS1yTsj0TJ+V7VybgVVK4v+nbz584guAPzvV4fJKa5QOSJ1VJrMLNhwBIApA0PpF9Y6EmQhhO1YkqvEC5JcCSFEXSS5Ei2iNVatrjR/TDd6BLlzscTI/351+KoTM9uyD3edISmzCG8XB54a10PtcIQQKugZUp1cncoqptJkVjkaIYSwP5JciebXiqtWFo56HcvujsVBp+G7o5l8dei82iG1qPSCMt7YegKAZydE4e0qU8KEaI86ejvj7qjHaDKTnF2sdjhCCGF3JLkSzS4lp7pq5dBKq1YW0SEe/GV0NwAWfX2UC/llKkfUcl78JpFSo4n+nby5s39HtcMRQqhEo9FYR7LLuishhKhNkivRvBSFPSmWqpVXq6xaXemhEV2IDfWiqLyKp7/8rV20B/6QlMXGwxnotBpemtwLrVajdkhCCBXJuqv2Ky0tjZEjRxIdHU2fPn34/PPP1Q5JCLsjyZVoVik5JWRdqlr16+SldjhNptdpWXp3DE4OWn46mcO/9qaqHVKzKq80sejrowDMGhJufVMlhGi/oq0TA4tUjkS0NL1ez/Lly0lMTGTz5s3Mnz+fkpIStcMSwq5IciWajdLGqlYWXfzdeHp89UCHJf89xpmctvuHZc2PyZy9WEqghyPzx3RTOxwhhB24chx7e6jei8uCg4OJjY0FICgoCD8/P3Jzc9UNSgg7I8mVaDbbjmVdUbVqvWut6jJzcDiDO/tSVmniic9/xWRue28wzuSU8I8fkgFYcGs07k4OKkckhLAHkUHuaDVwscRIdlH72ppix44dTJo0iZCQEDQaDRs2bKjX886fP899992Hr68vzs7O9O7dm/3791sfX7x4MRqNpsatRw/bTmWtb+yrVq0iPDwcJycnBg0axL59++o87sCBA5hMJkJDQ20apxCtnSRXolkoisLybdXT5WJCvXA26FSOyLa0Wg2v39UHN0c9+8/m8e5Pp9UOyaYURWHh10cxVpkZ3s2PW3oHqx2SEMJOODnoiPBzBeBoOxtqUVJSQkxMDKtWrar3c/Ly8hg6dCgODg58++23JCYmsnTpUry9a37o2LNnT9LT0623nTt3XvWcu3btorKystb9iYmJZGZmNjr2devWER8fz6JFizh48CAxMTGMGzeOrKysGsfl5uYyY8YM3n777Wu9dCHaJUmuRLPYdiyLI+cLq6tWbXSz2Y7eLiy8NRqApZtPkJTRdtYfbDqSwY4T2Rh0Wp6/rScajQyxEEJcdmVrYFsxatQooqOjr5l8TJgwgZdeeonbb7+93ud99dVXCQ0N5YMPPmDgwIFEREQwduxYunTpUuM4vV5PUFCQ9ebn51fn+cxmM3PnzmXq1KmYTCbr/UlJScTFxbF27dpGx75s2TIeeOABZs+eTXR0NGvWrMHFxYX333/fekxFRQWTJ0/mmWeeYciQIfX+PgjRXkhyJWyurVetrnTXgI6M7hGA0WQm/rMEjFWtf1PN4ooqnv9PIgAPjehMZ383lSMSQtib6JC2N9Ri+/btJCYmMnfuXJue9+uvv2bAgAHcddddBAQE0LdvX955551ax508eZKQkBA6d+7MtGnTSE2te2CSVqtl48aNHDp0iBkzZmA2m0lOTiYuLo7Jkyfz1FNPNSpOo9HIgQMHGDNmTI1rjRkzht27dwPVf99nzZpFXFwc06dPb9R1hGjrJLkSNrf1UtXK1aBrs1UrC41Gw8t39MbLxYGjFwpZ+f1JtUNqsje3nSSjsJxQH2ceHtVV7XCEEHaoLVaumsvp06dZvXo13bp147vvvuN//ud/ePTRR2tUmAYNGsSHH37Ipk2bWL16NSkpKQwfPpyiorqT15CQEL7//nt27tzJ1KlTiYuLY8yYMaxevbrRcebk5GAymQgMDKxxf2BgIBkZGUB1O+K6devYsGEDsbGxxMbGcvjw4UZfU4i2qG2MbxN2Q1EUlm+trlrNHBKOs+G4yhE1vwB3J16a3It5nxxi1Q/JjI4KJCbUS+2wGiUpo4j3dqYA8MJtvXByaLtVRyFE41nGsZ/OLqa80iS/K67BbDYzYMAAlixZAkDfvn05cuQIa9asYebMmUB1y55Fnz59GDRoEJ06deKzzz7j/vvvr/O8YWFhfPTRR4wYMYLOnTvz3nvvNXsL97BhwzCbW3+HhhDNSSpXwqa2Hsvi6IXqqtWfhndWO5wWc2ufECbFhGAyK8R/lkB5pen6T7IziqLw3IbDmMwK43oGMqpHgNohCSHsVIC7Iz6uBswKbWq9aXMIDg4mOjq6xn1RUVFXbfsD8PLyonv37pw6deqqx2RmZvLggw8yadIkSktLeeyxx5oUp5+fHzqdrtZAjMzMTIKCgpp0biHaE0muhM38vmrl42pQOaKW9eIfehLg7khydgmvf5ekdjgN9uXB8/xyJg9nBx0LJ/VUOxwhhB3TaDRXbCYsrYHXMnToUJKSav5NOHHiBJ06dbrqc4qLi0lOTiY4uO5JrTk5OYwePZqoqCjWr1/Ptm3bWLduHU888USj4zQYDPTv359t27ZZ7zObzWzbto3Bgwc3+rxCtDeSXAmb2ZKYaa1aPdCOqlYWXi4GXr2jDwDv70phz+mLKkdUf/mlRl7eeAyAv4zpRgcvZ5UjEkLYu6hgd6B9JVfFxcUkJCSQkJAAQEpKCgkJCdYq1MqVKxk9enSN5zz22GPs2bOHJUuWcOrUKT755BPefvvtGoMznnjiCX788UfOnDnDzz//zO23345Op2PKlCm1YjCbzUyYMIFOnTqxbt069Ho90dHRbNmyhQ8++IA33nijUbEDxMfH884777B27VqOHTvG//zP/1BSUsLs2bOb8m0Tol2RNVfCJqqrVtXDHGYNDce7nVWtLEb1CODeG0L59Jc0nvj8VzbNvwk3R/v/Z/bad0lcLDHSLcCNOUMj1A5HCNEKXB5q0X7aAvfv38+oUaOsX8fHxwMwc+ZMPvzwQ3JyckhOTq7xnBtuuIGvvvqKZ599lhdeeIGIiAiWL1/OtGnTrMecO3eOKVOmcPHiRfz9/Rk2bBh79uzB39+/VgxarZYlS5YwfPhwDIbLf2tjYmLYunVrnc+pT+wA99xzD9nZ2SxcuJCMjAxiY2PZtGlTrSEXQoirs/93faJV2JKYSWL6pbVWw9pf1epKz90azc5TOZzLK+Nv/03k5T/2UTukazqUmse/91V/cvni5F4Y9FLQFkJc35UTAxVFaRf74Y0cORJFUa76+OLFi1m8eHGt+2+99VZuvfXWqz7v008/bVAcN998c5339+3b96rPuV7sFvPmzWPevHkNikcIcZm8ixJNJlWrmtwc9fz9rhg0Gvj3vjS2H8+6/pNUYjIrPLfhCIoCf+zbgRs7+6odkhCileji74ZBp6WooopzeWVqhyOEEHZBkivRZFK1qu3Gzr7W9rqnv/yN/FKjyhHV7V97znL0QiHuTnqenRildjhCiFbEoNfSNaB6k/HEdrTuSgghrkXaAkWTSNWqbnkHl/NQoBmP0FTySoxs/HwLE3rXPfVJLSUVVZzfc4ZZPmZGRQagP3mMvCse9+43X63QhBCtRFSwB4nphRxLL2RcTxnXLYQQUrkSTbL5UtXKzVEvVavf0eu0jIsOQqPRcCKziJOZ9rXoe+fJbIxVZgI8nOjd0VPtcIQQrZBlYmDiBalcCSEESHIlmkBRFFZYqlZDpGpVl0BPJwaGewPw/fEsSiqqVI6o2rncUo5nFIEG4iID2sVCdCGE7UWHXBpqkSHJlRBCgCRXogmurFrdP0zGd1/NDRG++Ls7Ul5pYtuxTKjHtKbmZDIrfJ9UPWSjTwdPAj2dVI1HCNF6WTYSTssto6i8UuVohBBCfZJciUYxm69YayVVq2vSaTWM6xmETqshJadE9faZQ6l55JUYcTboGNLFT9VYhBCtm5eLgeBLH9Acz7Cv1mchhFCDJFeiUTYnZnLMstZquFStrsfXzZHBl8ac/3gym8IydT7hLSyrZG/KRQCGd/XH0UGnShxCiLbDst+V2h8cCSGEPZDkSjSY2aywYlt11Wr20HC8XKRqVR/9OnkT4uWMscrMlsTMem3maGs/nsimyqTQwcvZuhBdCCGaIvqKzYSFEKK9k+RKNNiVVStZa1V/Go2GsdGB6HVazuWV8mtafotePyW7mNPZxWg1GuJ6BIAMsRBC2ECUJFdCCGElyZVoEKlaNY2ni4HhXavXOe08lUNeSctsLlxlMvPDiWwA+oZ54ePm2CLXFUK0fZYq+PGMIqpMZpWjEUIIdUlyJRpkc2IGx9ILcZeqVaP16ehJmK8LJrPC5sQMzObmbw/8JSWXwrJK3J30DIrwbfbrCSHaj06+rjg76KioMnPmYona4QghhKokuRL1duWEQKlaNYFGw81RgRj0WjIKyjlwNq9ZL5dXYmR/avU1RnQPwEEv/+yFELaj02qIDLq0mXC6TAwUQrRv8i5L1NvmxAyOZxTh7qhnjlStmsTNyYFRkQEA7Em5SE5RRfNcSFHYnpSF2awQ7utKF3/X5rmOEKJds24mLOuuhBDtnCRXol6kamV7PYLc6eLvhtms8N3RDExm269VOJFZTFpuKTqthpGRMsRCCNE8ZKiFEEJUk+RK1Mt3R6VqZXMaDXFRATgbdOQUV7D3dK5NT19RZeLHk9VDLAaG++Dp4mDT8wshhEX0paEWsteVEKK9k+RKXJdMCGw+LgY9o3sEArD/bC7p+WU2O/ee5IuUVlTh5WKgXydvm51XCCF+LzKounKVVVTBxeJmanMWQohWQJIrcV1XVq3uH9ZZ7XDanC4BbkQFeaAo1XuIVdpglHF2UTm/nssHYFSkP3qd/FMXQjQfN0c94b4uAByToRZCiHZM3nGJa6pRtRoWIa1lzWREpD9ujnryS43sOpnTpHMpisL3x7NQFOgW6E6YrwyxEEI0P1l3JYQQklyJ69h0ZdVqqKy1ai6ODjpujq5uD/z1XD5puaWNPlfihUIyCspx0Gm5qZu/rUIUQohrsiRXiZJcCSHaMUmuxFWZzQortkrVqqWE+brSp6MnAFsSM6ioMjX4HGVGEztPVVe+Bnf2xc1Jb9MYhRDiaqRyJYQQklyJa9h0NIOkzCLcnaRq1VKGdfPH08WBovIqdiRlN/j5u07lUF5pws/NkZhQL9sHKIQQV2HZ6+pUVnGjPhwSQoi2QJIrUacrq1ZzhkrVqqU46LSMiw4CTXVrzens4no/Nz2/jKMXCgCI6xGAVit7WgkhWk6IpxMeTnqqzAqnsur/u0sIIdoSVZOrHTt2MGnSJEJCQtBoNGzYsKHG4+vXr2fs2LH4+vqi0WhISEiodY6RI0ei0Whq3B566KEax6SmpnLLLbfg4uJCQEAATz75JFVVVTWO+eGHH+jXrx+Ojo507dqVDz/80MavtnW5smol+1q1rGAvZ/qHVY9O33YskzLj9T8BNpurh1gA9AzxINjLuVljFEKI39NoNFe0BsrEQCFE+6RqclVSUkJMTAyrVq266uPDhg3j1VdfveZ5HnjgAdLT06231157zfqYyWTilltuwWg08vPPP7N27Vo+/PBDFi5caD0mJSWFW265hVGjRpGQkMD8+fP505/+xHfffWebF9rK1KpaOUvVqqXd2NkXXzcDpUYT249ngqJc8/hf0/LJKa7AyUHH0K4yxEIIoQ7rUAvZTFgI0U6putp9woQJTJgw4aqPT58+HYAzZ85c8zwuLi4EBQXV+djmzZtJTExk69atBAYGEhsby4svvsjTTz/N4sWLMRgMrFmzhoiICJYuXQpAVFQUO3fu5I033mDcuHGNe3Gt2LdHpGqlNv2l9sBPf0njZFYxSZlF1k06f6+4vIrdpy8CMLSrH84GXUuGKoQQVpZ1VzLUQgjRXrWJNVcff/wxfn5+9OrVi2effZbS0stjrHfv3k3v3r0JDAy03jdu3DgKCws5evSo9ZgxY8bUOOe4cePYvXv3Va9ZUVFBYWGh9VZU1DZaIKr3tToBwP3DpGqlJn8PJwZF+ACw/XgWxeVVdR6342Q2lSYzQZ5O9AypOwETQoiWEG1pC8woRLlOxV0IIdqiVp9cTZ06lX/9619s376dZ599lo8++oj77rvP+nhGRkaNxAqwfp2RkXHNYwoLCykrK6vzui+//DKenp7WW3R0tC1flmq+PZLBicxi3J30zJYJgaobEO5DoIcTFVVmth6r3R6YerGEk5lFaDQQFxmARiNDLIQQ6uka4IZOqyG/tJKMwnK1wxFCiBbX6pOrBx98kHHjxtG7d2+mTZvGP//5T7766iuSk5Ob9brPPvssBQUF1ltiYmKzXq8lSNXK/mi1Gsb1DESn1XD2YglHzl9utakymdmeVD3EIqajF/4eTmqFKYQQADg56Oji7wrIuishRPvU6pOr3xs0aBAAp06dAiAoKIjMzMwax1i+tqzTutoxHh4eODvXPXXN0dERDw8P683d3d2mr0MNG4+kS9XKDnm7OjK0ix/ergZcHXWYzAplxuoWweFd/QnxcubGLr4qRymEENVkM2EhRHvW5pIry7j24OBgAAYPHszhw4fJysqyHrNlyxY8PDysrXyDBw9m27ZtNc6zZcsWBg8e3DJB24ErJwRK1cr+xIZ5cfeAUDIKy3nnp2Te3nGad3eeJrOonMmxHXDUyxALIYR9iJZx7EKIdkzVaYHFxcXWChNUj0RPSEjAx8eHsLAwcnNzSU1N5cKFCwAkJSUB1ZWmoKAgkpOT+eSTT5g4cSK+vr789ttvPPbYY9x000306dMHgLFjxxIdHc306dN57bXXyMjI4LnnnmPu3Lk4OjoC8NBDD7Fy5Uqeeuop5syZw/fff89nn33Gf//73xb+jqhn45F0TmYV4yFVK7tkMiskpOazLyXXel9FpZl9Kblo0NA/3BsHXZv7rEQI0QpJ5UoI0Z6p+m5s//799O3bl759+wIQHx9P3759rXtQff311/Tt25dbbrkFgHvvvZe+ffuyZs0aAAwGA1u3bmXs2LH06NGDxx9/nDvuuIP//Oc/1mvodDq++eYbdDodgwcP5r777mPGjBm88MIL1mMiIiL473//y5YtW4iJiWHp0qW8++677WYMe82qVWepWtkhjUZDwrm8Oh9LOJeHVgZZCCHshCW5SrlYQqmx7imnQgjRVqlauRo5cuQ1R7XOmjWLWbNmXfXx0NBQfvzxx+tep1OnTmzcuPG6sRw6dOi652qL/nv4ctVq1tBwtcMRdTBWmaioNNf5WEWlGWOVCWeDqv+chRACAH93R/zcHMkpruB4RhH9wrzVDkkIIVqMvBtrIXkHl6sdQp0URSFlz1lm+Ri5sbMv5mPHqLs+At795rdkaOIKBr0ORwdtnQmWo4MWg6y5EkLYkegQD3acyOZYeqEkV0KIdkUWabRzJzOLyS0x4qjXEhvmpXY44ioURSG2Y91vUGI7emOWzTqFEHYkKrh6gq6suxJCtDdSuWrHFEVhb8pFAPqFecvEOTum12m5Ibw6uUo4l0dFpRlHBy2xHb25IdwbnQyzEELYEcvEQNnrSgjR3khy1Y5dWbWKkaqV3dPptPQP9+aGCB+MVSYMeh1mRZHESghhdyxDLY5nFGE2K2i1MnRHCNE+yLuydkqqVq2Tg06LTqvB2aBHp9XI+HUhhF3q7OeKQa+l1GgiNbdU7XCEEKLFyDuzdkqqVkIIIZqLXqclMlDWXbUmaWlpjBw5kujoaPr06cPnn3+udkhCtEqSXLVDUrUSQgjR3CxDLRIluWoV9Ho9y5cvJzExkc2bNzN//nxKSkrUDkuIVkfWXLVD1qqVg06qVkIIIZqFZd2VVK5ah+DgYIKDgwEICgrCz8+P3NxcXF1d/397dx4WVd2+Afw+rKKg4IbihiiKuIGA5paCuKK9mvnTMlFTSdNM3EJNpXJP1Nz3tbc0d01zwy1TcwuXcEHcF0DcAEGBmef3B++cmMQyHRhnuD/X5XXFOWdmni8zDec+3+UYuTIi08Keq3xGRHBU12tVzpG9VkRElCs81XCVbORKDOfgwYNo164dXFxcoCgKNm3a9FKPu337Nj788EMUK1YMdnZ2qFmzJk6cOJHntc2ZMweurq4oUKAA6tWrh2PHjuX4XCdPnoRGo0G5cuUMWiNRfsBwlc9cik/GQ/ZaERFRLvP4X7i6/SgNj1MzjFyNYTx58gS1a9fGnDlzXvoxDx8+RMOGDWFtbY2ff/4Z0dHRiIiIgJNTzvcu/PXXX5GR8fzvKzo6GvHx8a9c25o1azB48GCMHTsWp06dQu3atdGyZUskJCToHffgwQMEBwdj4cKFL91GIvoThwXmI1lzrR4AYK8VERHlriJ21ijjaIfbj9IQfTcJ9SsVM3ZJr61169Zo3br1v3rM5MmTUa5cOSxbtkzdVrFixRyP1Wq16N+/P9zd3bF69WpYWmb9nb548SICAgIwePBgDB8+/JVqmzZtGvr06YOePXsCAObPn49t27Zh6dKlCAsLAwA8e/YM7du3R1hYGBo0aPCv2klEWdhzlY9k77XyYq8VERHlMlOad+Xv7w9PT89/1Sv1MrZs2QJfX1906tQJJUuWhLe3NxYtWpTjsRYWFti+fTt+//13BAcHQ6vVIjY2FgEBAWjfvv0Lg9U/SU9Px8mTJxEYGKj3WoGBgThy5AiArAuwPXr0QEBAALp16/ZKr0NEDFf5RvZeK5/yjrBhrxUREeUyTxfTCVf79u1DdHQ0+vfvb9DnvXLlCubNmwd3d3fs3LkT/fr1w8CBA7FixYocj3dxccHevXtx6NAhfPDBBwgICEBgYCDmzZv3yjUkJiZCo9HA2dlZb7uzszPi4uIAZA1HXLNmDTZt2gQvLy94eXnh7Nmzr/yaRPkVhwXmE3pzrco5GrscIiLKBzz/txz7+bg3P1zlFq1WC19fX0yYMAEA4O3tjXPnzmH+/Pno3r17jo8pX748Vq1ahSZNmsDNzQ1LliyBoii5WmejRo2g1Wpz9TWI8gP2XOUD7LUiIiJj0A0LvBSfggxN/jxxL126NDw9PfW2VatWDTdu3HjhY+Lj4xESEoJ27dohNTUVoaGhr1VD8eLFYWlp+dyCGPHx8ShVqtRrPTcR6WO4ygd0vVYF2GtFRER5qJxTQRSysUR6phZX7uXPG9I2bNgQFy9e1Nt26dIlVKhQIcfjExMT0axZM1SrVg0bNmxAZGQk1qxZg6FDh75yDTY2NvDx8UFkZKS6TavVIjIyEvXr13/l5yWi5zFcmTm9FQLLO7HXioiI8oyFhaIuyW4K867+SUpKCqKiohAVFQUAuHr1KqKiotReqNmzZ6NZs2Z6jwkNDcXRo0cxYcIEXL58Gd9//z0WLlyY49wurVaL1q1bo0KFClizZg2srKzg6emJ3bt3Y9myZZg+ffor1zZ48GAsWrQIK1aswPnz59GvXz88efJEXT2QiAyDc67M3KW47L1WRYxdDhER5TOepQvj5PWHOH83Ce29yxi7nNdy4sQJ+Pv7qz8PHjwYANC9e3csX74ciYmJiI2N1XuMn58fNm7ciBEjRuCrr75CxYoVMWPGDHTt2vW557ewsMCECRPQuHFj2NjYqNtr166NPXv2oESJEq9cW+fOnXHv3j2MGTMGcXFx8PLywo4dO55b5IKIXg/DlRljrxURERmbbt5VtBn0XDVt2hQi8sL94eHhCA8Pf25727Zt0bZt25d6jebNm+e43dvb+7VqA4ABAwZgwIABL1UHEb0aDgs0Y5fikvEwlb1WRERkPNV0KwaaQbgiIvonDFdmir1WRET0JqhaygGKAiSmpCMh+amxyyEiylUMV2bqol6vlaOxyyEionyqoI0VKhYvBAA4fzfZyNUQEeUuhiszpNUKjun1WvFtJiIi46lmRisGEhH9HZ51m6FL8ey1IiKiN4enblGLOwxXRGTeGK7MDHutiIjoTcNFLYgov+CZt5lhrxUREb1pPEtnrVh7JfEJnmZojFwNEVHuYbgyI1rtnysE+lRgrxUREb0ZnAvbwqmgNTRaQUx8irHLISLKNTz7NiOx91Lw6H+9VrXKOhq7HCIiIgCAoijZbib82MjVEBHlHoYrE5ep0UKjFaSmZ6Ji8UJoV8sFDSsXY68VERG9Uf5cMZDLsROR+bIydgH06jQaLU5ce4ioWw/xLEMLW2sL1C7rCD/XosYujYiISI+6YiAXtSAiM8ZwZaIy/xesfrt6X932LEOLY1cfQIECH1cnWFuy94qIiN4M2e91JSJQFMXIFRERGR7Pvk2UoiiIuvUwx31Rtx7Cgn+0iIjoDVK5pD2sLRUkP83ErYdpxi6HiChXMFyZqPRMDZ5laHPc9yxDi/RMLnVLRERvDhsrC1QqYQ+A97siIvPFcGWibKwsYWud89tna20BGyvLPK6IiIjo73lyUQsiMnMMVyZKROBV1inHfV5lnaAVyeOKiIiI/p6ny5/zroiIzBEXtDBRVpYW8HPNClfZVwv0KusEP1cnWHIxCyIiesNU44qBRGTmGK5MmKWlBXxcneBXsSjSMzWwsbKEVoTBioiI3ki6cHXjQSqSn2bAoYC1kSsiIjIsnoWbOGtLC1haKLCzsYKlhcLl14mI6I1VtJANnAvbAgAuxnHeFRGZH56JExERUZ7xLM15V0RkvhiuiIiIKM/8Oe+KPVdEZH4YroiIiCjPcFELIjJnDFdERESUZ3Th6mJcEjRa3jaEiMwLwxURERHlmYrFC6GAtQWeZmhx7f4TY5dDRGRQDFdERESUZywtFFQtxUUtiMg8MVwRERFRnvIs7QAAiL7DcEVE5oXhioiIiPJUNS7HTkRmiuGKiIiI8tSf97ricuxEZF4YroiIiChPefwvXMUlPcWDJ+lGroaIyHAYroiIiChP2dtaoXzRggA4NJCIzAvDFREREeW5av9b1ILhiojMCcMVERER5TnP0kUAANEMV0RkRhiuiIiIKM/92XPFRS2IyHwwXBEREVGe0y3HfjkhGemZWiNXQ0RkGAxXRERElOfKOtnBoYAVMjSCywkpxi6HiMggGK6IiIgozymKgmqleDNhIjIvDFdERERkFJ4uDFdEZF4YroiIiMgodItacMVAIjIXDFdERERkFLpFLc7fTYKIGLkaIqLXx3BFRERERlHF2QEWCvAwNQPxSc+MXQ4R0WtjuCIiIiKjKGBtiUol7AFw3hURmQejhquDBw+iXbt2cHFxgaIo2LRpk97+DRs2oEWLFihWrBgURUFUVNRzz/H06VP0798fxYoVg729PTp27Ij4+Hi9Y27cuIGgoCAULFgQJUuWxLBhw5CZmal3zP79+1GnTh3Y2tqicuXKWL58uYFbS0RERH+lGxrIeVdEZA6MGq6ePHmC2rVrY86cOS/c36hRI0yePPmFzxEaGoqtW7di7dq1OHDgAO7cuYN3331X3a/RaBAUFIT09HQcPnwYK1aswPLlyzFmzBj1mKtXryIoKAj+/v6IiorCoEGD0Lt3b+zcudNwjSUiIqLnVCtdGEUL2eBRarqxS6F/cPPmTTRt2hSenp6oVasW1q5da+ySiN44VsZ88datW6N169Yv3N+tWzcAwLVr13Lc//jxYyxZsgTff/89AgICAADLli1DtWrVcPToUbz11lvYtWsXoqOjsWfPHjg7O8PLywtff/01Pv/8c4SHh8PGxgbz589HxYoVERERAQCoVq0aDh06hOnTp6Nly5aGbTQRERGp3qntgu4NKuDBk3SkZ2qRqdWioI1RT0/oBaysrDBjxgx4eXkhLi4OPj4+aNOmDQoVKmTs0ojeGCY95+rkyZPIyMhAYGCgus3DwwPly5fHkSNHAABHjhxBzZo14ezsrB7TsmVLJCUl4Y8//lCPyf4cumN0z5GTZ8+eISkpSf2XnJxsyKYRERGZvWcZGvx44ibemhiJRpP3wXf8biw4cAXPMjTGLo1yULp0aXh5eQEASpUqheLFi+PBgwfGLYroDWPS4SouLg42NjZwdHTU2+7s7Iy4uDj1mOzBSrdft+/vjklKSkJaWlqOrz1x4kQUKVJE/efp6WmIJhEREeULaemZmLs/Ft9GxiApLWsedFJaJr6NjMHc/bFITc/8h2fIe/80Vzwn4eHhUBRF75+Hh8e/Piavap8zZw5cXV1RoEAB1KtXD8eOHcvxuJMnT0Kj0aBcuXIGrZPI1Jl0uDKmESNG4PHjx+q/6OhoY5dERERkMiwtLLDs8NUc9y07fBVWFm/eKco/zRV/kerVq+Pu3bvqv0OHDr3SMTq//vorMjIyntseHR393KJe/6b2NWvWYPDgwRg7dixOnTqF2rVro2XLlkhISNA77sGDBwgODsbChQtf+FxE+ZVJD2ouVaoU0tPT8ejRI73eq/j4eJQqVUo95q9XXXRfPNmP+euXUXx8PAoXLgw7O7scX9vW1ha2trbqz0lJXOWIiIjM18NTMwz6fGnpmXjX7gqQ859ZJJ44CzsDz71yqjPohfv8/f1hbW2N/v37o3///jke809zxV/EyspKPed4nWMAQKvVon///nB3d8fq1athaWkJALh48SICAgIwePBgDB8+/JVqnzZtGvr06YOePXsCAObPn49t27Zh6dKlCAsLA5A1LaJ9+/YICwtDgwYN/rFeovzmzbss9C/4+PjA2toakZGR6raLFy/ixo0bqF+/PgCgfv36OHv2rN5Vl927d6Nw4cLqUL769evrPYfuGN1zEBERkWHZWFnC1jrn0xBbawvYWFnmaT379u1DdHT0C4PV64iJiYGLiwvc3NzQtWtX3Lhx45WOAQALCwts374dv//+O4KDg6HVahEbG4uAgAC0b98+x2D1MtLT03Hy5Em9OegWFhYIDAxU56CLCHr06IGAgAB10TEi0mfUcJWSkoKoqCj1/lVXr15FVFSU+oXy4MEDREVFqUPuLl68iKioKHWuVJEiRdCrVy8MHjwY+/btw8mTJ9GzZ0/Ur18fb731FgCgRYsW8PT0RLdu3XD69Gns3LkTX3zxBfr376/2PPXt2xdXrlzB8OHDceHCBcydOxc//vgjQkND8/g3QkRElD+ICLzKOuW4z6usE7QieVxR7qhXrx6WL1+OHTt2YN68ebh69SoaN26stxDWyxyTnYuLC/bu3YtDhw7hgw8+QEBAAAIDAzFv3rxXrjMxMREajSbHOei6865ff/0Va9aswaZNm+Dl5QUvLy+cPXv2lV+TyBwZdVjgiRMn4O/vr/48ePBgAED37t2xfPlybNmyRe2aBoAuXboAAMaOHYvw8HAAwPTp02FhYYGOHTvi2bNnaNmyJebOnas+xtLSEj/99BP69euH+vXro1ChQujevTu++uor9ZiKFSti27ZtCA0NxbfffouyZcti8eLFXIadiIgol1hZWsDPNStcRd16iGcZWthaW8CrrBP8XJ1gaWnSg2tU2Yfi1apVC/Xq1UOFChXw448/olevXi99zF+VL18eq1atQpMmTeDm5oYlS5ZAUZRcbUujRo2g1Wpz9TWITJ1Rw1XTpk0hf3NlqkePHujRo8ffPkeBAgUwZ86cv52gWaFCBWzfvv0fa/n999//9hgiIiIyHEtLC/i4OsGvYlGkZ2pgY2UJrYjZBKucODo6okqVKrh8+fJrHRMfH4+QkBC0a9cOx48fR2hoKGbNmvXKdRUvXhyWlpY5zkF/mblgRJTFfL+9iIiI6I1nbWkBSwsFdjZWsLRQYG3GwQrImhIRGxuL0qVLv/IxiYmJaNasGapVq4YNGzYgMjISa9aswdChQ1+5LhsbG/j4+OjNQddqtYiMjOQcdKJ/wby/wYiIiIgM5J/mis+ePRvNmjXTe8zQoUNx4MABXLt2DYcPH0aHDh1gaWmJ999//18do6PVatG6dWtUqFABa9asgZWVFTw9PbF7924sW7YM06dPf6XagazpGYsWLcKKFStw/vx59OvXD0+ePNGbokFEf8+kl2InIiIiyiv/NFc8MTERsbGxeo+5desW3n//fdy/fx8lSpRAo0aNcPToUZQoUeJfHaNjYWGBCRMmoHHjxrCxsVG3165dG3v27MnxMS9TOwB07twZ9+7dw5gxYxAXFwcvLy/s2LHjuUUuiOjFGK6IiIiIXsI/zRUPDw9XF9zSWb169T8+78sck13z5s1z3O7t7f3Cx/xT7ToDBgzAgAED/lU9RPQnDgskIiIiIiIyAIYrIiIiIiIiA2C4IiIiIiIiMgCGKyIiIiIiIgNguCIiIiIiIjIAhisiIiIiIiIDYLgiIiIiIiIyAN7nykC0Wi0A4O7duznuT3vwOC/LyRVPbt36149hu00X2/3y2G7TxXa/PHNtt+7vtu7vOBHR61DkZe4oR//o+PHjqFu3rrHLICIioldw7Ngx+Pn5GbsMIjJxDFcGkpmZid9//x3Ozs6wsMjb0ZbJycnw9PREdHQ0HBwc8vS1jYntZrvzA7ab7c4PjNlurVaL+Ph4eHt7w8qKA3qI6PUwXJmBpKQkFClSBI8fP0bhwoWNXU6eYbvZ7vyA7Wa784P82m4iMj9c0IKIiIiIiMgAGK6IiIiIiIgMgOHKDNja2mLs2LGwtbU1dil5iu1mu/MDtpvtzg/ya7uJyPxwzhUREREREZEBsOeKiIiIiIjIABiuiIiIiIiIDIDhioiIiIiIyAAYroiIiIiIiAyA4YqIiIiIiMgAGK6IiIiIiIgMgOHKhOSHVfO1Wm2O2/ND24nIvImI+l2W377ToqKikJmZCSD/tZ2I8heGKxMQFRWFlJQUKIpi7FJylVarhYVF1kfy/PnzuHz5MmJjYwHA7NtO+RtPNs2T7mKRLlQoioIrV66o/51fpKamwt/fH0OHDgWQv9pORPkPw9UbbvTo0QgNDcVPP/0EjUZjtidhIqIGq5EjR6Jjx45o0KABmjdvjpEjRxq5utyne1/v37+PhISEHPeZG1277t69i/Pnz+Px48fIyMgA8OIeTHOga/fNmzcRHR2Nmzdv4unTp0auKvfp2h0bG4vLly/j+vXrz+0zNxYWFrh06RKGDRsGAFi7di1q1qyJixcvGrmy3JW9hw4AChYsiG+//RYHDx7Erl27jFgZEVHuszJ2AfRiI0eOxKJFi/D999/Dx8cHlpaW6r7svTzmQHcl85tvvsGCBQuwZs0aaDQaXL9+HYMHD0ZiYiIWLlxo5Cpzj6Io2LhxIyZOnIi4uDi0b98ewcHB8PX1haIoEBGzutqra8+mTZswevRoJCYmonz58mjdujUGDhyIokWLmt1nHPiz3Rs3bsTnn38OKysrPHv2DE2bNkXfvn3h5+dn7BJzjaIoWLduHQYPHozMzEy4ubkhODgYISEhZvkZ17l16xa+/fZbnD9/Hnv27MGSJUtQtWpVs20v8Of3+bFjx1C1alUUKVIEzZs3x8aNG7F9+3a8/fbbKFCggJGrJCLKJUJvpOPHj0uVKlXkyJEjIiLy+PFjuXTpkixYsEBiYmJERESj0RizxNd28+ZNvZ/T09OlQ4cO8vXXX+tt37Vrl1hbW8u3336bl+XlqRMnToizs7OMHTtWIiIixM3NTdq1aye7du1Sj9FqtUas0PC2b98uDg4OEhERIfHx8RIaGiouLi4SEhIi9+7dExHT/4znZP/+/eLg4CAzZ84UEZHp06eLtbW1LF261MiV5Q7d5/b27dtSqVIlWbx4saxfv14GDRok5cuXl6lTpz53rLnQtWf06NGiKIr4+/vLkydPjFxV3rh06ZIoiiLVq1eXhQsXyt27d+XgwYNiZ2cnO3fuFBHz/P+biMi8LgubkQIFCuDZs2dIS0vDuXPn8MUXXyAoKAiTJk1CzZo1cenSJZO+qt+rVy8MHDhQb1tGRgb++OMP3L9/X92m0WjQvHlz9OnTB/v27UN6errZDSG6fPky9u/fj08++QTh4eEYPHgwNm3ahPj4eMycORN79uwBAPXqvin65Zdf9H5OSEjAtGnTMGrUKAwePBhWVlZYv349ypUrh8OHD+OLL77AgwcPYGFhYTZDBHXt+Omnn9CpUyd8+umnuH37NmbNmoVevXqhZ8+eAIDHjx8bs0yDUxQFR44cwaxZs9C6dWv07NkT7777LoYMGYIePXpg+vTpiIiIUI811c94TjQaDQDA0tISgwcPxsGDBzFo0CDcvXs3x+PNqe3FihWDn58fbt26hevXr6Nr166wsLDAsGHD8OmnnyIuLs6k/4YREb0Iv9neMBcuXEBqaiqcnJxQs2ZN9O/fH/Xq1UNGRga+/vprHD9+HBUrVsSOHTuMXepriYiIwJo1awAADx8+BJA1Lv+DDz7AwYMHcfToUQBQh0IWKVIESUlJsLa2NpuhNCKCBw8eICAgAF988YXeCVfNmjWxaNEixMfHY/bs2di2bRsA05wIvnfvXgQFBSExMVE9eSxWrBi6d++Od955B/fu3UODBg3Qpk0bHD16FD4+PlizZg369++PxMREkzwByx4IdfPIdG1/8uQJ6tSpg8ePH6Nu3boIDAzE3LlzAQCbN2/Gnj171JNyc5CSkoKVK1di3rx5OH/+vPp+li1bFn369EGvXr0we/ZsjBs3DoBpfsb/Svde6xayGDNmDKZOnYrNmzdj6dKlGDt2LOLj49XjT5w4AcA82n779m0kJCSgaNGiWLhwIcqUKYMiRYogODgYLVu2xLlz55CcnIxFixYhPT3d2OUSERmc6Z21mLENGzbAz88P27ZtQ5kyZRAREYGJEydi27ZtmDlzJjp37gw7OzvY29ujRIkSxi73tTg6OsLa2hpLlixB9erVERMTAwDw9/dH4cKFMXv2bBw+fBhA1pX8EydOwM3NzSxOPoA/594ULVoUK1euRNmyZXH69Gn1JAsAatWqhSVLliA6OhqrVq1CamqqESt+dQ0aNMDly5dRvHhx3LhxA0BWaH733XdRrVo1LF++HJUrV8b48eMBAHXq1EHp0qWRkZFhsidfFhYWuHXrFgDA2toa27dvx+LFiwFkffYnT56MGjVq4L333sOcOXOgKAoyMjKwbt06HD9+3Cx663QBw97eHn369MH777+PQ4cOYdWqVeoxZcuWRUhICDp27IgffvgBDx48MPneG93/27t27ULfvn3Rpk0bhISE4OrVqwgKCsK2bduwbNkyjBkzBqdOncK4ceMQGBiod/HBVF24cAFBQUEYPnw4Tp48idq1ayM4OBh37txBcHAw9u3bBwcHB8THx2Pjxo3qhQciIrNilMGI9EL/+c9/xNXVVdatWydPnz5Vt6elpcmVK1ckKChIfH19JTMz04hVGk58fLx4eXlJzZo15fLlyyIisnHjRmnevLk4OzuLr6+vuj89PV1ETHtehq523funm3MQGRkprq6u8uGHH8qpU6f0HnPu3Dm5cuVK3haaC65cuSKKosg333yjt33IkCFSt25dSUlJUX+ePHmy3L9/3xhlGkRycrL4+vpKYGCgrFu3ThRFkbVr14qISFJSkrRo0UIcHR0lOTlZRLLmG44YMULKlCkjFy9eNGbpr033GU9LS5OMjAx1+6VLl6RPnz7i4eEh3333nd5jbt++rc6zMwcbN24UOzs7GTVqlEydOlUaN24shQsXlri4OBER2bNnjxQtWlRq164tzs7OcuLECSNXbDgzZ86UDz/8UBwcHGTJkiXy3//+V5o3by67d+8Wkazv/N27d6vf90RE5obh6g3UsWNHKV++vKxdu1ZSU1NFRGTu3LnSunVradCggRoyTC1gvWjycmJiotSpU0c8PDwkNjZWREQuXLggmzdvlpEjR8q8efPUk7TsJ2umRnfSuWfPHunfv7907dpVxo8fL/Hx8SIisnPnTnF1dZWuXbtKVFSUMUvNFRkZGRIeHi42NjZ6i5PMmTNH6tatKx07dpRu3bpJoUKF5NKlS0as9PU9ffpUIiMjpWTJkmJrayurVq0SkawQpdVqJTIyUqpXry6lS5eWZs2aScuWLaVkyZLPBWtTo/uMb9u2TVq1aiUNGzaUNm3ayLFjx0REJCYmRkJCQqRq1ary/fffG7PUXHP//n1p0KCBTJ8+XUREbty4IeXKlZPevXuLyJ+/o/Pnz8uBAweeW9jHlOjakpqaqncxJDU1VWbMmCGlSpWSTz/9VNzd3aVmzZomfcGEiOhlMVwZ2Zw5c2Tfvn3PBY8OHTpIqVKlZN26dSIicubMGVm1apUaqEwtZGRv39GjR2XHjh1y5swZefjwoYhknZDoAtaLrmiaWpjMycaNG6VAgQLSu3dvad68ufj6+kqFChXk+vXrIpK1MqK7u7u88847cubMGSNX+3p0J17R0dHyyy+/yI0bN0REZOrUqaIoisyYMUNEsoLI2LFj5d1335XWrVubZLtzunBw7do1cXJyEkdHR/nPf/6jt0+r1cqDBw9k/PjxMmzYMJkxY4Z6YcHU/fTTT2JtbS1Dhw6VcePGSZMmTcTFxUUNmH/88Yd88sknUrJkSfnxxx+NXK3h3bhxQ8qXLy+3bt2Su3fvSpkyZSQkJETdv2bNGvV7z5Tp/v/eunWrBAYGioeHhwQFBcmqVavUi4IHDhyQTz/9VOrXry+KosiXX35p0iMPiIheBsOVkXl5eUnZsmXl119/fe4EzcvLS2rVqiUrV67U+4NkSiFDq9Xq1T58+HBxcXERV1dXsbGxkS5duqjL8t6/f198fHzE09PT5HsucnLv3j3x8vKSKVOmqNvOnj0rLVq0kIoVK0pCQoKIiOzYsUNq164tt2/fNlapBrNx40axt7eXSpUqia2trSxatEji4+Nl2rRpoiiKREREiMif4SQtLc2Y5b6W27dvy/Hjx0VE5LvvvpMhQ4ZIbGys7Nq1SypVqiRt2rRRjzW1iyMvQ6vVSkpKijRr1kzCwsL09vXs2VNcXFzk9OnTIiJy8uRJCQ0NNauhYb/99ps8ePBAUlNTpU2bNrJo0SIpX768fPzxx+r7fePGDenatats377dyNUaxvbt28XGxkbCwsJk/vz50qJFC6lbt66MGjVKDVgJCQly+PBh8ff3l/Pnzxu5YiKi3MdwZST//e9/1WExAQEB4urqKocOHVKDk1arlc6dO4ujo6O89957xiz1lf11uMuCBQukRIkScuDAAXn48KH89NNP0qJFCwkKCpKDBw+KSFYA8fX1lRo1asidO3dExHzuhXL16lVxdnbWu3dVZmamREVFSZ06dWTWrFlqW039XjgajUbu378vDRs2VO/NNm7cOFEURSZNmiR3796VadOmia2trUyYMMHY5b4WrVYraWlp4ufnJ//5z39k4sSJoiiKLFiwQESyeuY2bdoklStXlrZt26qPmzt3rixatEg0Go3JXc3PXm/2oKjRaMTb21sd9pl93mj9+vWlU6dO6s/Pnj3Lg0pzh25+oM7ly5elTJkycuHCBdFoNNK5c2dRFEWvvSJZF5dM9cJJYmKiiGS99xqNRp48eSJt27bVC9Lp6ekycuRI8fPzM8teSSKil8FwZQTnzp0Tb29vqVWrlmzbtk1ERJo2bSqurq5y8OBB9cS6Z8+eEh0dbZLh4pNPPpFhw4aJyJ89bR999JEEBwfrHbd//37x8fFRjxXJ6sHy9vaWhg0b5l3BeeDp06dSp04dGTVqlN52rVYrdevWlQEDBuhtM0XZFzNITU2VkSNHyoMHD9T9M2bM0AtY48ePl6JFi+odY6piYmKkXLlyoiiKjB07Vm9f9oBVq1Yt6du3ryiKIn/88Ydxin0NuvdYd7ItInLw4EH59ddfRUTE399fWrVqpe7TBaxBgwbphUtTNXfuXKlRo4ZeQIqNjZWKFSuq8yeTk5PF29tbvLy8ZPLkybJy5Ur5+OOPpXDhwiY5n/K7776TZs2aPfd5bdq0qQwePFhE/vyez8zMlLfffls6d+6c53USEb0JuBR7Hhs2bBjGjh0LOzs73Lp1C59++ik2b96Mffv2wd3dHd27d0eHDh3QoEEDHD16FFWqVDHJG6m2aNFCXVr70aNH6vbk5GQAf94HqEmTJnj//fexZMkSPHr0CCKCokWLYtKkSbh//z5iY2PzvHZDkGz3uUlLSwMA2NraomnTpti9ezc2bNigHqsoCsqUKQNHR0dI1gUPk11yXlEUbN68Ge3bt4evry82bNiAmzdvqvs/++wzTJ8+HWPGjMH8+fPRq1cvxMTEwMnJyYhVv76MjAwULVoUlpaWKFmyJC5evIjffvtN3W9ra4s2bdpg1apVqFq1KhISEnD69Gl4enoasepXoygKEhMT0bZtW0yYMAFbtmxBkyZNkJSUBAAYMWIEYmJi1JuE29raAgASExPh4OCAzMxMk15yvHnz5khOTsYHH3yAO3fuAMi6XYS1tTWKFSsGjUYDe3t77Nu3D56envjxxx8xefJkxMfH49ChQ6hdu7aRW/DydO9TSkoK0tPTER4ejvPnzwMAnj17hiJFiiA6OhpA1u0VNBoNLC0t0bJlS9y8edNkb6VARPRajJns8ptly5aJo6OjnDx5Uh48eCB3796VFi1aiK+vr2zZskVERCZNmiQff/yxfPLJJ+pwG1Pqufprj8uKFSukZcuWcvv2bVm7dq0oiiK//PKL3jFr1qyRt956S5KSktRtU6dOFXd3d72r46Yi+4ppXbp0kWrVqsnQoUMlMjJSnj59Km3btpW33npLPvvsM/nxxx9lwIABUrhwYbOYj3D8+HEpXLiw9O3bV3r06CHW1tby2WefybVr1/SOmzhxojg5OZnk+/t3kpOT5fTp01K5cmV599135ciRIzkep5uPYmp0dV+/fl3GjRsnZcuWlQIFCsgPP/ygHpOUlCSzZs2SihUrir+/v4wePVq6d+8uhQoVkrNnzxqrdIO6evWqVKpUSRo1aiRxcXFy+PBh8fDwyHEu3ZMnTyQ5OVlviKSpyP75Xblypfj7+0uHDh3U9/HMmTNiZ2cnAwcO1Ps7FRwcLB07dlRXtiUiyk8YrvLQqFGjpFGjRqLRaNQ/RLdu3ZJ69eqp97bS0e039Ynvc+fOlfr168sHH3wgt2/fls8++0yKFCki27Ztk2vXrsnDhw+lefPm0rZtW71gtmHDBpMMG7o2bN68WQoWLCijR4+WVatWSdOmTaVixYpqm4cPHy5vvfWWuLu7S+PGjeX33383buEGcPnyZRkzZoxMnDhR3TZ37lwpW7ashIWFPRewTH0ooO69joqKkh9++EFWrlyptvG3336TypUrS6dOneTw4cMiIjJixAgJDw/Xe6wpWblypTg7O6v3atqxY4coiiLOzs5677lIVsg8cOCAtG3bVpo1aybvvfee2QQrHV3AatGihWzZskW8vLxk5syZsm7dOtm3b59s3LhRli5dKufOnTN2qa9k7969UqJECb33dsWKFWrA0q3quX79erGzs5OAgADp0aOHdO/eXezt7U1y1U8iIkNguMoDuhOpr776Snx9fdUV0XRX9fbu3SsFCxYUf39/Wb16tdHqzC3Lly+Xxo0by//93//JsWPHJCwsTOzs7KRs2bLi6ekpXl5eJnvvLpGsHirdKmharVYSEhKkSZMm6n1uUlNTpUSJEhIaGvpcL2R8fPxzk+NN0ePHj8XX11eKFy8uI0eO1Ns3e/ZsKVOmjIwaNUrvZsimGDD+at26dVKuXDnx8/OTJk2aiLW1tWzdulVERE6cOCHVq1eXt956S1q0aCEFCxZ8YU/Wm0z3Pu3bt0/q168vNWrUkISEBLl//75s2bJFJk6cKB4eHs/NM8vOHHowdL+HCxcuqBd+rl69KpUrVxZFUaR69epSq1YtqVGjhvj4+Ii7u7tUrVpVLly4YMyyX9mFCxdk6NChUr16dZk8ebK6fcWKFdK0aVPp0KGDREdHi0jW7RaCg4OlQ4cOEhwcbHZBmojo32C4ykNnzpwRS0tL9eq1zo4dO6Rjx44SEBAggYGBJr2KVnbZT56XLl0qTZo0kc6dO8uDBw/k9OnTsnbtWlm7dq3J3rtLRCQuLk4qVqyoLj4ikrWSmI+Pj1y4cEGuXLkiZcqUkT59+qiP2bFjh8TExBir5Fxz6tQpcXd3l4YNGz53cjVv3jwpUKCAfPnllyb5Pufk+PHjUrRoUVm4cKGIiFy8eFEURZGvvvpKDdFRUVEycuRIGTBggEkuXiGSda8ikaz/n48cOSKNGjVSA5ZI1v28xo4dKx4eHvLll1+qj/vuu+/URS5MPUjr6l+/fr14enrKmDFj5O7duyKSFbB8fHykZs2aEhsbKxkZGZKRkaEu6mJqpk6dql7wuX79unz++edStWrVFwYs3YUlXYA2hyBNRPQ6GK7y2LJly8Ta2lqGDRsmJ06ckNjYWAkKCpLx48dLdHS0KIoiu3fvNnaZBvPXgNWoUSPp3Lmz2oOh22+KPVY6J0+eFD8/P+ndu7ecPXtW0tLSxNPTU+bPny+VK1eW3r17q+27evWqdOnSxWzuc/NXp0+fFi8vLwkJCXluONTixYtN9v5lulsFZLd27Vp1qe0rV65I2bJlpV+/fup+3bDHzMxMk5o3mV1kZKQ4ODioq+BptVo5dOiQNGrUSDw8PNQhgtevX5fw8HDx8PCQjz76SEaNGiWKopjVfax27twpdnZ2Mm/ePDVY6ly9elXc3NzE399fvVm2KYqJiZHGjRvrDcmOiYn524DVqVMnvRUQTT1IExG9LoYrI1i3bp2ULFlSypYtK2XKlBFvb29JS0uTa9euibu7u3ol0Fz8NWA1btxYL2CZg1OnTkmdOnWkV69ecufOHZk9e7YoiqJ341gRkZEjR0qNGjVM+gTsn+h+F7179zbZ3prsdAHj3r17ep/ladOmScOGDSU2NlbKly8vISEhaojasmWL9O/f3+SHfKalpanBKvsFEV3Aqlatmhqwbt68KbNmzZK6detK/fr15dSpU0ar25C0Wq08ffpUunXrJkOGDFG3iehfFLp27Zo4OTlJq1atTPZiUWZmpiQnJ4uIyC+//KL+94sC1qpVq8Tb21u6detmNiMuiIheF8OVkdy6dUuOHDkiBw8eVE/IwsLCxMPDQx1uYk6yn5QuX75c3n77bfn888/l6dOnZnOl89SpU+Ll5SW9e/eWPXv2yNChQ8XS0lKmTJkiU6ZMkX79+omDg4NZLF7xT06dOiV169aVLl26mOTCJNllDxjZF+U4duyYNGnSRJycnKRHjx4i8udCNKGhofJ///d/8vjx47wvOBdcuXJFFEWRKVOmiMiLA9bTp09Fo9HIo0ePjFlurmjQoIGEhobmuE93seT69esmO+Q3+/fwnTt3xN/fXypXrqxeIHhRwPrhhx+eW6yGiCg/Y7h6A5w7d066desmxYoVM+sT7+x/vIcOHSqNGjUyu6udp06dEh8fH/n4449l165dMmvWLKlevbrUq1dP3n///Xw10VsXPu7cuWPsUgxCFzC++eYbERF59uyZ9O7dW0qVKiWzZ8+WlJQUuXXrloSFhUmxYsVMdpW4nGRkZEh4eLjY2NioC7VkD1g1a9Y0u4tC2XunUlJSJDAwUD744AN1m+6YmzdvSlhYmNkMgTx16pSEhITI2rVrpUGDBlKnTp3nAlb16tWfmztMRERZGK6MLCMjQ06dOiVDhgwxq5OxF9GdsISHh4ubm5tZXuE+efKk+Pr6Su/eveXOnTtqm3WrROYn5tTm7AFj2rRpIpLVvo4dO0rNmjXF3t5eGjRoIJUqVTL5IXG6z2x0dLT88ssvas/M1KlTRVEUvYD166+/So0aNaRevXomO7csO13bnzx5IiKi3p9q69ateuFaZ8SIEeLr66v2bpq6adOmiY+Pj5w4cUIOHToktWvXFh8fH72ANWDAAPHz85PExESzGXlARGQoisj/bsFORpWRkQFra2tjl5EnRATr1q1DlSpVULt2bWOXkyt+//13fPzxx3Bzc8Po0aNRvXp1iAgURTF2afSSdO/X+fPncf/+fVSoUAHlypVDREQEhg0bhm+++QZDhgxBeno6zp8/j6ioKFSuXBmurq4oU6aMsct/bZs2bUK3bt3g7OyMW7duYfbs2XjnnXfw3//+F0OGDMG0adMwaNAgaLVaHD9+HM7OznB1dTV22a9F957//PPPWLhwIR4/fozixYtj5MiR8PLywsyZMzFo0CB06NAB9vb20Gg02Lp1K/bv3w9vb29jl/9KdG1OS0uDnZ0dAKBx48YoUKAAdu/ejaNHj6Jfv36wtLTEgQMHUKhQIVy5cgX29vYoWbKkkasnInoDGTHYEZk1cxsWlx9t3LhR7O3tpVKlSmJrayuLFi2S+Ph4mTZtmiiKIhEREcYu0eA0Go3cv39fGjZsKAsWLJCYmBgZN26cKIoikyZNkrt378q0adPE1tZWJkyYYOxyDW7z5s1iY2MjYWFhMnDgQGnTpo0UKFBAvX9ZZGSkfPjhh9KuXTvp27evWSzasmPHDvnwww9l586dIpI1d8zNzU0mTZokIlnL8detW1fc3NxMfpEWIqLcxnBFlIvMaVhcfvJvAobuBNTUZR++mpqaKiNHjlSXkxcRmTFjhl77x48fL0WLFpX79++bzdCw9PR0ad26tYSFhanbUlNTZdCgQWJraytnzpwREVHniprDPdu0Wq306dNHFEWRokWLytixY+XKlSsyfvx4ee+99+TMmTOi1Wplx44d0rRpU7Na5ZWIKDdYGbvnjMicFShQwNgl0L8g/xsilZ6eDjs7OzRp0gSdOnWCk5MTRo0aBXt7e4SGhgIAunfvjrS0NEyZMgUhISFwcnIycvWvR1EUbN68GfPmzcPNmzeh1WrRuXNntV2fffYZAGD48OFIS0tDv3790LdvXxQtWtSYZRvM5s2bERMTg2vXruGdd94BkPV5sLW1xbhx43Du3DnMnDkTc+fOhYWFBQDA0tLSmCW/Msk2RFlRFPTu3RspKSmoUaMGNm7ciPj4eGRmZuKPP/7A7t27UbNmTQQEBKBx48YoWLCgkasnInqzWRi7ACKiN4UuYLRv3x6+vr7YsGEDbt68qe7/7LPPMH36dIwZMwbz589Hr169EBMTY/LBCgBOnDiB4OBgVKxYEXXr1kVsbCyWLl2K69evq8d89tln+PLLLzFz5kxYWVmZTbA6efIkPvroI1SoUAFVqlTBtm3bkJqaCkVRoCgKChUqhFKlSiExMRHW1tawssq6LmmqcygVRcHevXuxePFiAICvry+KFSuG2NhY7N27F7Vq1QIAXLhwAUOHDsWRI0dgbW3NYEVE9BIYroiI/ie/BozY2Fhs3boVI0aMwLx587Bs2TJ8++23WL9+PebPn6/X/rCwMMTGxqJYsWJGrNhwLl++jC1btqBPnz7o1KkTWrdujXv37iEiIgJPnz5VA5SVlRWcnJyQkZEBMfF1oDQaDX777TeEhIQgODgYR44cwcyZMxEVFYUZM2agX79+iIiIQP/+/eHi4oISJUoYu2QiIpPB1QKJiJAVMFauXAk7OzuEhYUBAObNm4cJEybgww8/RN++fVGhQgX1+IcPH5pFj1VSUhKaNWuGa9euISQkBOPHj1f3zZkzBxMnTkSPHj3Qq1cvVKxYEQDMZuVLXduvX7+Orl27Yvr06cjMzMSIESNw4MABODg4IDAwEBcvXsT69etx9OhRVK9e3dhlG8yZM2cwbNgwpKSkwM/PD61atcL8+fMxfPhwNGjQAADw6NEjODo6GrdQIiITwp4rIsr3kpKS0KVLF8ydOxfJycnq9n79+iEsLAyrVq3CokWLcPXqVXWfuZxwFi5cGAsXLoSTkxMOHDiAc+fOqfv69++PL774AhEREVi1ahUyMzMBmO5wuL/Std3R0RH79+/HqVOnYGVlhUmTJmHAgAEoVaoUNm3ahJSUFBw+fNisghUA1KpVCytXrkTfvn1x4MABvPfeezhz5gy2b9+uHmMun3MiorzCnisiImTdm6xz584oWbIk5s+fjxo1aqj75s+fj9DQUIwYMQIjR45U59yYkzNnzqB79+6oW7cuBg4cqBcklixZgrfffhvu7u5GrDD3nDlzBt26dUPdunXx6aefqnOOACAtLQ1WVlZmfx/CjIwMfP7555g9ezacnJxw+fJlODg4GLssIiKTw3BFRPQ/+TlgAFkBs3fv3qhTpw5CQ0Ph6elp7JLyTPa2Dxo0yOx6qf5O9mGee/bsgbu7u94QWCIienkMV0RE2eTngAFktb9v375wc3PD2LFj4eHhYeyS8kx+bru5zKMjIjI2zrkiIsrG29sbixcvxpkzZ/D111/jwoULxi4pT3l7e2P27Nm4e/cuihQpYuxy8lR+bjuDFRGRYbDniogoB8ePH8ewYcPwww8/oHTp0sYuJ889ffo0394EOz+3nYiIXg/DFRHRC/Akm4iIiP4NhisiIiIiIiID4JwrIiIiIiIiA2C4IiIiIiIiMgCGKyIiIiIiIgNguCIiIiIiIjIAhisiIiIiIiIDYLgiIiIiIiIyAIYrIiIiIiIiA2C4IiIiIiIiMgCGKyKibDQaDbRarbHLICIiIhPEcEVEbzxXV1fMmDFDb5uXlxfCw8MhIggPD0f58uVha2sLFxcXDBw4UD3u2bNnGDp0KMqUKYNChQqhXr162L9/v7p/+fLlcHR0xJYtW+Dp6QlbW1vcuHEDrq6umDBhAj766CM4ODigfPnyWLhwoV4Nn3/+OapUqYKCBQvCzc0No0ePRkZGhro/PDwcXl5eWLp0KcqXLw97e3t88skn0Gg0mDJlCkqVKoWSJUti/Pjxes/76NEj9O7dGyVKlEDhwoUREBCA06dPG+4XSkRERLnCytgFEBG9jvXr12P69OlYvXo1qlevjri4OL0gMmDAAERHR2P16tVwcXHBxo0b0apVK5w9exbu7u4AgNTUVEyePBmLFy9GsWLFULJkSQBAREQEvv76a4wcORLr1q1Dv3790KRJE1StWhUA4ODggOXLl8PFxQVnz55Fnz594ODggOHDh6uvHxsbi59//hk7duxAbGws3nvvPVy5cgVVqlTBgQMHcPjwYXz00UcIDAxEvXr1AACdOnWCnZ0dfv75ZxQpUgQLFixAs2bNcOnSJRQtWjSvfrVERET0LykiIsYugojo77i6umLQoEEYNGiQus3Lywvt27dH4cKFsWDBApw7dw7W1tZ6j7tx4wbc3Nxw48YNuLi4qNsDAwNRt25dTJgwAcuXL0fPnj0RFRWF2rVr671m48aNsWrVKgCAiKBUqVL48ssv0bdv3xzrnDp1KlavXo0TJ04AyOq5+uabbxAXFwcHBwcAQKtWrXDx4kXExsbCwiJr8ICHhwd69OiBsLAwHDp0CEFBQUhISICtra363JUrV8bw4cMREhLyGr9JIiIiyk3suSIik9apUyfMmDEDbm5uaNWqFdq0aYN27drBysoKZ8+ehUajQZUqVfQe8+zZMxQrVkz92cbGBrVq1XruubNvUxQFpUqVQkJCgrptzZo1mDlzJmJjY5GSkoLMzEwULlxY7zlcXV3VYAUAzs7OsLS0VIOVbpvueU+fPo2UlBS9+gAgLS0NsbGx/+ZXQ0RERHmM4YqI3ngWFhb4aye7bm5TuXLlcPHiRezZswe7d+/GJ598gm+++QYHDhxASkoKLC0tcfLkSVhaWuo93t7eXv1vOzs7KIry3Ov+tSdMURR1sYsjR46ga9eu+PLLL9GyZUsUKVIEq1evRkRExD8+x989b0pKCkqXLq03L0zH0dHxuW1ERET05mC4IqI3XokSJXD37l3156SkJFy9elX92c7ODu3atUO7du3Qv39/eHh44OzZs/D29oZGo0FCQgIaN25s0JoOHz6MChUqYNSoUeq269evv/bz1qlTB3FxcbCysoKrq+trPx8RERHlHYYrInrjBQQEYPny5WjXrh0cHR0xZswYtSdq+fLl0Gg0qFevHgoWLIjvvvsOdnZ2qFChAooVK4auXbsiODgYERER8Pb2xr179xAZGYlatWohKCjolWtyd3fHjRs3sHr1avj5+WHbtm3YuHHja7c1MDAQ9evXR/v27TFlyhRUqVIFd+7cwbZt29ChQwf4+vq+9msQERFR7uBS7ET0xhsxYgSaNGmCtm3bIigoCO3bt0elSpUAZA2VW7RoERo2bIhatWphz5492Lp1qzpnadmyZQgODsaQIUNQtWpVtG/fHsePH0f58uVfq6Z33nkHoaGhGDBgALy8vHD48GGMHj36tduqKAq2b9+Ot99+Gz179kSVKlXQpUsXXL9+Hc7Ozq/9/ERERJR7uFogERERERGRAbDnioiIiIiIyAAYroiIiIiIiAyA4YqIiIiIiMgAGK6IiIiIiIgMgOGKiIiIiIjIABiuiIiIiIiIDIDhioiIiIiIyAAYroiIiIiIiAyA4YqIiIiIiMgAGK6IiIiIiIgMgOGKiIiIiIjIAP4fKXpU04SpQawAAAAASUVORK5CYII=\n", - "text/plain": [ - "
" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "matplotlib.rc_file_defaults()\n", - "ax1 = sns.set_style(style=None, rc=None )\n", - "\n", - "fig, ax1 = plt.subplots()\n", - "plt.xticks(rotation=45, ha='right')\n", - "\n", - "\n", - "sns.lineplot(\n", - " data=df, x='username', y='total_upvotes',\n", - " marker='o', ax=ax1, label=\"Sum of upvotes\")\n", - "ax1.get_legend().remove()\n", - "\n", - "ax2 = ax1.twinx()\n", - "sns.barplot(data=df, x='username', y='num_posts',\n", - " order=df['username'], alpha=0.5, ax=ax2, log=True,\n", - " color=\"orange\", label=\"Number of posts\")\n", - "\n", - "\n", - "# ask matplotlib for the plotted objects and their labels\n", - "lines, labels = ax1.get_legend_handles_labels()\n", - "lines2, labels2 = ax2.get_legend_handles_labels()\n", - "ax2.legend(lines + lines2, labels + labels2, bbox_to_anchor=(1.55, 1))" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "You should see a correlation between total number of upvotes and total number of posts. In order to track user impact on a more equal footing, normalize the total number of upvotes relative to the total number of posts, and plot the result:" - ] - }, - { - "cell_type": "code", - "execution_count": 15, - "metadata": {}, - "outputs": [ - { - "data": { - "image/png": "iVBORw0KGgoAAAANSUhEUgAAAkAAAAHMCAYAAAA9ABcIAAAAOXRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjcuMCwgaHR0cHM6Ly9tYXRwbG90bGliLm9yZy88F64QAAAACXBIWXMAAA9hAAAPYQGoP6dpAACLeElEQVR4nO3dd1xT5/cH8E/YIFNkyhQRRFFQHDjqVlyto1ZbrVpXbbXO2or9ujocbW3VDm1t1VpbW2frqHvvCU5EBGSDiuxNcn5/8MstKaAEEi5Jzvv1yktzc3PvuQGSk+c5z/NIiIjAGGOMMaZD9MQOgDHGGGOsrnECxBhjjDGdwwkQY4wxxnQOJ0CMMcYY0zmcADHGGGNM53ACxBhjjDGdwwkQY4wxxnSOgdgB1EcymQzJycmwsLCARCIROxzGGGOMVQMRIScnB87OztDTe34bDydAlUhOToarq6vYYTDGGGOsBhISEuDi4vLcfTgBqoSFhQWAshfQ0tJS5GgYY4wxVh3Z2dlwdXUVPsefhxOgSsi7vSwtLTkBYowxxjRMdcpXuAiaMcYYYzqHEyDGGGOM6RxOgBhjjDGmczgBYowxxpjO4QSIMcYYYzqHEyDGGGOM6RxOgBhjjDGmczgBYowxxpjO4QSIMcYYYzqHZ4JmjLE6IpVJcTb+LFJyUuBk4YSubl2hr6cvdliM6SROgBhjrA7sjtiNmYdmIjE7UdjmYumCNSFrMKz5MBEjY0w3cRcYY4yp2e6I3Xh1+6sKyQ8AJGUn4dXtr2J3xG6RImNMd3ECxBhjaiSVSTHz0EwQqMJj8m2zDs2CVCat69AY02mcADHGmBqdjT9boeWnPAIhITsBZ+PP1mFUjDFOgBhjTI1SclJUuh9jTDU4AWKMMTVysnBS6X6MMdXgBIgxxtSoq1tXuFi6QAJJpY9LIIGrpSu6unWt48gY022cADHGmBrp6+ljTciaSh+TJ0WrQ1bzfECM1TFOgBhjTM2GNR+GHwf/WGG7rZktdr62k+cBYkwEnAAxxlgdICob8t68UXN0du0MAJjebjonP4yJhGeCZoyxOrA/aj8AYLT/aJgYmOB8wnncTLspclSM6S5OgBhjTM0KSgpwNPooAGBQs0FIL0gHAISlhokZFmM6jRMgxhhTsxOxJ1BQWgBXS1e0cmiFzMJMAMCjzEfIKMiAjamNuAEypoO4BogxxtRs/4Oy7q/BzQZDIpHAxtQGHtYeAIDw1HDxAmNMh4maAHl4eEAikVS4TZs2DQDw448/onv37rC0tIREIkFmZuYLj7lkyZIKx/P19VXzlTDGWOWISKj/GdRskLA90DEQAHeDMSYWUROgq1evIiUlRbgdPVrWRz5ixAgAQH5+PkJCQrBgwQKljtuiRQuF4547d07lsTPGWHWEp4YjMTsRDQwboIdnD2E7J0CMiUvUGiA7OzuF+ytWrICXlxe6desGAJg1axYA4NSpU0od18DAAI6OjtXev6ioCEVFRcL97Oxspc7HGGNVkXd/9fHqAxMDE2F7oNP/J0ApnAAxJoZ6UwNUXFyMrVu3YsKECZBIKp8yvrqioqLg7OyMJk2aYPTo0YiPj3/u/suXL4eVlZVwc3V1rdX5GWNMbt+DfQCAQd6DFLbLW4DuP72PgpKCOo+LMV1XbxKgv/76C5mZmRg/fnytjtOhQwds3rwZhw4dwrp16xAbG4uuXbsiJyenyueEhoYiKytLuCUkJNQqBsYYA4DU3FRcTb4KABjYbKDCY84WzrAzs4OUpLj9+LYY4TGm0+rNMPiff/4Z/fv3h7Ozc62O079/f+H/rVq1QocOHeDu7o7t27dj4sSJlT7H2NgYxsbGtTovY4z914EHBwAA7ZzbwdFcsVteIpEg0CkQR6KPICwlDO0btxcjRMZ0Vr1oAYqLi8OxY8cwadIklR/b2toazZo1w8OHD1V+bMYYex5599fgZoMrfZwLoRkTT71IgDZt2gR7e3sMHDjwxTsrKTc3F9HR0XByclL5sRljrCqFpYU4GlM2snWwDydAjNU3oidAMpkMmzZtwrhx42BgoNgjl5qaivDwcKH15vbt2wgPD8ezZ8+EfXr16oVvv/1WuP/+++/j9OnTePToES5cuIChQ4dCX18fr7/+et1cEGOMATgZexL5JflwsXRBa4fWle4jHwl2K+0WSmWldRkeYzpP9ATo2LFjiI+Px4QJEyo8tn79egQGBmLy5MkAgJdeegmBgYHYu3evsE90dDSePn0q3E9MTMTrr78OHx8fvPbaa7C1tcWlS5cqDLlnjDF1Kj/6q6qRrU0bNoW5kTkKSwtx/+n9ugyPMZ0nISISO4j6Jjs7G1ZWVsjKyoKlpaXY4TDGNAwRwX21OxKyE3DgjQMY4D2gyn27buqKc/HnsGXIFrzZ+s06jJIx7aPM57foLUCMMaZtbqXdQkJ2AkwNTNHDo8dz9+U6IMbEwQkQY4ypWPnZn00NTZ+7LydAjImDEyDGGFOxqmZ/roy8EDo8NRxckcBY3eEEiDHGVCgtNw1Xkq4AUFz9vSp+dn4w1DNEZmEmHmU+UnN0jDE5ToAYY0yF/on6BwRCkHMQnCxePP+Ykb4RWtq3BMDdYIzVJU6AGGNMhZTp/pIT6oB4ZXjG6gwnQIwxpiKFpYU4En0EQNWzP1dGXgfELUCM1R1OgBhjTEVOPzqNvJI8OFs4C6061cEjwRire5wAMcaYilRn9ufKtHZsDQkkSM5JRlpumrrCY4yVwwkQY4ypABEJ8/8o0/0FAOZG5vC29QbArUCM1RVOgBhjTAXuPL6DuKw4mBiYoKdnT6Wfz4XQjNUtToAYY0wF5N1fvZv0hpmhmdLPb+PUBgC3ADFWVzgBYowxFRC6v5op1/0lx4XQjNUtToAYY6yWHuc9xqXESwCAgd4Da3QM+VD4h88eIrsoW2WxMcYqxwkQY4zVknz25zZObdDYsnGNjtHIrBFcLF0AADdTb6oyPMZYJTgBYoyxWqpt95ccd4MxVnc4AWKMsVooKi3C4ejDAKq3+OnzcALEWN3hBIgxxmrhdNxp5BbnwsncSRjJVVPCkhg8FJ4xteMEiDHGakHe/TXQeyD0JLV7S5W3AN19chdFpUW1jo0xVjVOgBhjrIaISJj/R9nZnyvjZuUGGxMblMpKcffJ3VofjzFWNU6AGGOshu49uYdHmY9gYmCC3k161/p4EomEu8EYqyOcADHGWA3JW396evas0ezPleFCaMbqBidAjDFWQ0L3Vy2Hv5cnT4BupNxQ2TEZYxVxAsQYYzXwNP8pLiZcBFD74e/lybvAbqbdhFQmVdlxGWOKOAFijLEakM/+HOAYIMzgrAo+tj4wNTBFfkk+op5Fqey4jDFFnAAxxlgNqKP7CwD09fTR2rE1AC6E1kZSmRSnHp3CttvbcOrRKW7lExEnQIwxpqRiaTEOP1TN7M+V4UJo7bQ7Yjc81nigxy898MbuN9Djlx7wWOOB3RG7xQ5NJ3ECxBhjSjoTdwY5xTlwNHdEkHOQyo/PCZD22R2xG69ufxWJ2YkK25Oyk/Dq9lc5CRIBJ0CMMaYkVc7+XJnycwERkcqPz+qWVCbFzEMzQaj4s5Rvm3VoFneH1TFOgBhjTAnlZ39WR/cXALS0bwl9iT7SC9IrtBgwzXM2/uxzf44EQkJ2As7Gn63DqBgnQIwxpoSIpxGIyYiBsb4x+jTpo5ZzmBiYwM/ODwB3g2mDlJyUau33zZVvcDbuLIqlxWqOiAGcADHGmFLk3V89PXuigVEDtZ2Hl8TQHk4WTtXab3fEbry0+SXYrLRByNYQfHH+C1xPvs5dY2oiagLk4eEBiURS4TZt2jQAwI8//oju3bvD0tISEokEmZmZ1Trud999Bw8PD5iYmKBDhw64cuWKGq+CMaZL1N39JceF0Nqjq1tXuFi6QAJJpY9LIIGNiQ1G+I2AnZkd8kvycTj6MD449gGCNgTB7gs7DPtzGL678h0inkRwXZiKGIh58qtXr0Iq/TezvXPnDvr06YMRI0YAAPLz8xESEoKQkBCEhoZW65h//vkn5syZg/Xr16NDhw5YvXo1+vXrh8jISNjb26vlOhhjuiE9Px0XEi4A4ASIVZ++nj7WhKzBq9tfrfCYPCn66eWfMKz5MBAR7jy+gxOxJ3A89jhOx51GRmEG9tzfgz339wAAnC2c0dOzJ3p69ESvJr3gZuVWp9ejLSRUj1LJWbNmYf/+/YiKioJE8m+mfOrUKfTo0QMZGRmwtrZ+7jE6dOiAdu3a4dtvvwUAyGQyuLq64r333sP8+fOrFUd2djasrKyQlZUFS0vLGl8PY0y7bL21FW/ueROtHFrh5tSbaj1XVmEWrFdaAwCeznsKWzNbtZ6Pqd/uiN0YuWMkSqlU2OZq6YrVIasxrPmwSp9TKivF9eTrOB57HCdiT+B8wnkUlhYq7ONl44Venr3KkiLPnrBrYKfW66jPlPn8FrUFqLzi4mJs3boVc+bMUUh+lD3G9evXFVqL9PT00Lt3b1y8eLHK5xUVFaGoqEi4n52dXaPzM8a0m7pmf66MlYkVmtg0QUxGDMJSw9C7SW+1n5OpV1e3rkLys2HQBjS1bYqubl2hr6df5XMM9AzQwaUDOrh0wIKuC1BYWoiLCReFhOhK0hVEZ0QjOiMaP974EQDQyqGV0Dr0kvtLsDTmL/KVqTcJ0F9//YXMzEyMHz++xsd4+vQppFIpHBwcFLY7ODjg/v37VT5v+fLlWLp0aY3PyxjTfiXSEhx6eAhA3SRAANDGqU1ZApTCCZA2uJhY9kXcz84Pk9pOqtExTAxM0MOzB3p49gAAZBdl40zcGaHL7FbaLeG2+vJq6Ev00a5xOyEh6uTaCSYGJiq7Jk2mVAIUERGBP/74A2fPnkVcXBzy8/NhZ2eHwMBA9OvXD8OHD4exsXGNAvn555/Rv39/ODs71+j5tREaGoo5c+YI97Ozs+Hq6lrncTDG6q+z8WeRXZQN+wb2aNe4XZ2cM9AxEDvv7eQ6IC0hrx/r5NJJZce0NLbEoGaDhJq0J3lPcPLRSSEhevjsIS4lXsKlxEtYdm4ZjPWN0dmts5AQBTkHwUCv3rSF1KlqXfWNGzfwwQcf4Ny5c+jcuTM6dOiAoUOHwtTUFM+ePcOdO3fw0Ucf4b333sMHH3yAWbNmKZUIxcXF4dixY9i9u3ZTgTdq1Aj6+vpIS0tT2J6WlgZHR8cqn2dsbFzjxI0xphv2RZZ1f6lr9ufKcCG0dhESIFfVJUD/ZdfADq+1eA2vtXgNABCfFS8kQ8djjiMlNwUnYk/gROwJ/O/k/2BhZIFuHt2EGqKW9i3r7PdbbNVKgIYPH4558+Zh586dzy1CvnjxItasWYNVq1ZhwYIF1Q5i06ZNsLe3x8CBA6v9nMoYGRmhbdu2OH78OIYMGQKgrAj6+PHjmD59eq2OzRjTXeVnf66r7i/g37mAIp9GIq84T63zDjH1KpYW42ryVQDqTYD+y83KDeMDxmN8wHgQESLTI4WE6GTsSWQUZmD/g/3C/FZ2Znbo4dkDvTx7oZdnLzSxaVLjutz6rloJ0IMHD2BoaPjC/YKDgxEcHIySkpJqByCTybBp0yaMGzcOBgaK4aSmpiI1NRUPHz4EANy+fRsWFhZwc3NDw4YNAQC9evXC0KFDhQRnzpw5GDduHIKCgtC+fXusXr0aeXl5eOutt6odE2OMlReZHonojGgY6Ruhj5d6Zn+ujKO5IxzNHZGam4pbabcQ7BpcZ+dmqnUz9SYKSwvR0LQhmtk2EyUGiUQC30a+8G3ki3fbvQupTIqbaTdxPOY4Tjw6gTNxZ/Ak/wm2392O7Xe3AyhLoMqPMHO2qPsyFXWpVgJUneSnpvsfO3YM8fHxmDBhQoXH1q9fr1Cc/NJLLwEoazGSF0tHR0fj6dOnwj4jR47EkydPsGjRIqSmpiIgIACHDh2qUBjNGGPVJe/+6uHRA+ZG5nV67kDHQBx8eBBhqWGcAGmw8t1f9aVFRV9PH22c2qCNUxvM6zwPxdJiXEm6guMxx3E89jguJV5CfFY8NoVvwqbwTQAA30a+QutQN49uaGjaUOSrqLlqzQO0du3aah9wxowZtQqoPuB5gBhj5XXb3A1n4s7gm/7fYHr7uu1O/+j4R1h2bhkmBU7Chpc31Om5meqM3DkS2+9ux7KeyxDatXoT+4otrzgP5+LPCV1mN1JuKKxoL4EEbZzaoKdnT/Ty7IUubl2q1U0rlUlxNv4sUnJS4GTh9MKpAJShzOd3tRIgT09PhftPnjxBfn6+UA+UmZkJMzMz2NvbIyYmpuaR1xOcADHG5J4VPIP9F/aQkhSPZj6Cu7V7nZ5/572dGLFjBNo6tcW1Kdfq9NxMdVy/dkVidiJOjTuFbh7dxA6nRjIKMnDq0SlhDqKIpxEKjxvqGaKjS0ehy6yDSwcY6Rsp7LM7YjdmHpqJxOxEYZuLpQvWhKypcjJIZag8ASrv999/x/fff4+ff/4ZPj4+AIDIyEhMnjwZb7/9NkaPHl3zyOsJToAYY3K/3/4do3ePhr+9P269c6vOzx/9LBpNv2kKI30j5IbmwlBfuZIEJr6ErAS4rXaDvkQf2aHZMDM0EzsklUjOScbJ2JNlI8xijyM+K17hcTNDM3R16yokRLEZsXht52sKrUjAv8uB7HxtZ62TILUmQF5eXti5cycCAwMVtl+/fh2vvvoqYmNjlY+4nuEEiDEm9/qu1/HHnT8Q2iUUy3otq/Pzy0gGm5U2yC7Kxs2pN9HKoVWdx8Bq5887f2LUrlEIcg7C1clXxQ5HLYgIMRkxQuvQidgTeJL/RGEfCSQVkp/yj7lYuiB2ZmytusOU+fxWerB/SkoKSktLK2yXSqUV5t9hjDFNViItwcGogwDqdvh7eXoSPQQ4BgAAwlJ4PiBNJC+ADnbR3iJ2iUQCr4ZemNJ2Cv549Q+kvp+KW1Nv4et+X2Nws8EwNTCtMvkBAAIhITsBZ+PP1lnMSidAvXr1wttvv40bN24I265fv4533nkHvXvzVO2MMe1xPuE8soqyYGdmh/aN24sWB0+IqNkuJKp/AsT6Rk+iB38Hf8zqOAt7X9+LHwf/WK3npeSkqDmyfymdAG3cuBGOjo4ICgoSZlBu3749HBwc8NNPP6kjRsYYE4V8+PsA7wEqG6VSE/IE6EbKjRfsyeqbvOI8oeVOlxKg/3KxdKnWfk4WTmqO5F9KLwBiZ2eHf/75Bw8ePBAWGPX19UWzZuJM7MQYY+oixuzPlWnj1AYAEJ4aDhnJdGapAm1wLfkapCRFY4vGcLXU3TUmu7p1hYulC5KykyrtCpPXAHV161pnMdX4r8jDwwM+Pj4YMGAAJz+MMa3zIP0Bop5FwVDPsE5nf66MbyNfGOsbI6c4BzEZmj/ViC6pjxMgikFfTx9rQtYA+HfUl5z8/uqQ1XXa0qp0ApSfn4+JEyfCzMwMLVq0QHx82bC39957DytWrFB5gIwxJgZ591d3j+6wNBZ3NKihviH8HfwBcCG0ptHF+p+qDGs+DDtf24nGlo0VtrtYuqhkCLyylE6AQkNDcfPmTZw6dQomJibC9t69e+PPP/9UaXCMMSaW/VFli0OK3f0lx4XQmoeIcDHhIgBOgOSGNR+GRzMf4eS4k/h92O84Oe4kYmfG1nnyA9SgBuivv/7Cn3/+iY4dOyo057Vo0QLR0dEqDY4xxsSQUZCBs3Flw3EHNRskcjRlOAHSPFHPopBekA4TAxNhKgNW1h3W3aO72GEo3wL05MkT2NvbV9iel5en0/2bjDHtcejhIUhJihZ2LeBp4/niJ9SBQKf/T4C4C0xjyOt/2jm3q7AkBBOf0glQUFAQDhw4INyXJz0//fQTgoO1d5InxpjuqG/dXwDQyqEV9CR6SMtLq9O5UljNlS+AZvWP0l1gy5YtQ//+/XHv3j2UlpZizZo1uHfvHi5cuIDTp0+rI0bGGKszpbJSYfbn+tL9BZStq+Rj64OIpxEISw2r0/lSWM1wAlS/Kd0C1KVLF4SHh6O0tBT+/v44cuQI7O3tcfHiRbRt21YdMTLGWJ05H38eGYUZsDW1RUeXjmKHo4C7wTRHZmEm7j65C0C7l8DQZEq3AAFlC6Ju2LBB1bEwxpjo9j8o6/4Se/bnygQ6BuL3279zIbQGuJR4CQDQtGFT2DWwEzkaVhmlW4B69uyJpUuXVtiekZGBnj17qiQoxhgTS32Z/bkyPBJMc3D3V/2ndAJ06tQpfPvttxgyZAjy8vKE7cXFxVwDxBjTaFHpUYhMj4SBngH6Ne0ndjgVyLvAYjJikFmYKW4w7LmEBMiFE6D6qkZLYRw7dgypqano2LEjHj16pOKQGGNMHPLur27u3USf/bkyDU0bwt3KHUDZumCsfiqVleJy0mUA3AJUn9UoAXJycsLp06fh7++Pdu3a4dSpUyoOizHG6l597v6S40Lo+u/O4zvILc6FpbEl/Oz8xA6HVUHpBEg+74+xsTF+//13zJw5EyEhIfj+++9VHhxjjNWVzMJMnI2vX7M/V4brgOo/efdXR5eO9a6Qnv1L6VFgRIrL2P/vf/9D8+bNMW7cOJUFxRhjde3ww8MolZWieaPm8GroJXY4VeIEqP67mPj/639x/U+9pnQCFBsbi0aNGilsGz58OHx8fHD9+nWVBcYYY3VJE7q/gH+7wCKeRKCgpACmhqYiR8T+i0eAaQalu8Dc3d2hp1fxaS1btuRWIMaYRiqVleLgw/o3+3NlGls0RiOzRpCSFHce3xE7HPYfqbmpiMmIgQQSdHDpIHY47Dmq1QI0bNgwbN68GZaWlhg27PlL1u/evVslgTHGWF25mHARzwqeoaFpQwS71u9ZeyUSCQIdA3E05ijCUsPQrnE7sUNi5VxMKOv+8nfwr5cjCdm/qpUAWVlZCcXPVlZWag2IMcbqmrz7a4D3ABjo1WiC/DolJEA8Eqze4fl/NEe1/tI3bdpU6f8ZY0wbyOf/GeRdv7u/5ISh8FwIXe9cSCxLgOp7SyKr4TxAjDGmLaKfRSPiaQQM9AwQ0jRE7HCqRT4S7FbaLUhlUpGjYXJFpUW4lnwNABdAa4JqtQAFBgYKXWAvcuPGjVoFxBhjdUne+vOS+0uwMtGMLn5vW280MGyAvJI8RKZH8mR79cSNlBsolhbDzswOXjb1dyoFVqZaCdCQIUPUHAZjjIlDXv+jKd1fAKAn0UNrx9a4kHABYSlhnADVE+WHv1e30YCJp1oJ0OLFi9UdB2OM1bmswiycjitbxHmwT/2e/+e/2ji2wYWEC7iRcgOjW40WOxyGf+t/uPtLM3ANEGNMZx2JPoJSWSl8bH3QtGFTscNRChdC1y9ExBMgahilx3tKpVJ8/fXX2L59O+Lj41FcXKzw+LNnz1QWHGOMqZOmzP5cmfJLYhARd7mILC4rDqm5qTDUM0Rbp7Zih8OqQekWoKVLl+Krr77CyJEjkZWVhTlz5mDYsGHQ09PDkiVLlDqWh4cHJBJJhdu0adMAAIWFhZg2bRpsbW1hbm6O4cOHIy0t7bnHHD9+fIXjhYRoxsgOxljdkcqk+CfqHwCa1/0FAC3sW8BQzxCZhZmIy4oTOxydJ2/9aePUhpcn0RBKJ0C//fYbNmzYgLlz58LAwACvv/46fvrpJyxatAiXLl1S6lhXr15FSkqKcDt69CgAYMSIEQCA2bNnY9++fdixYwdOnz6N5OTkF85EDQAhISEKx922bZuyl8kY03KXEi8hvSAdNiY2GtllYaRvhBb2LQCAJ0SsB7j7S/Mo3QWWmpoKf39/AIC5uTmysrIAAIMGDcLChQuVOpadnZ3C/RUrVsDLywvdunVDVlYWfv75Z/z+++/o2bMngLJJGJs3b45Lly6hY8eOVR7X2NgYjo6O1Y6jqKgIRUVFwv3s7GylroMxpnnk3V/9vftrxOzPlQl0DER4ajjCUsMwtPlQscPRaZwAaR6lW4BcXFyQkpICAPDy8sKRI0cAlLXmGBsb1ziQ4uJibN26FRMmTIBEIsH169dRUlKC3r17C/v4+vrCzc0NFy9efO6xTp06BXt7e/j4+OCdd95Benr6c/dfvnw5rKyshJurq2uNr4Mxphnk8/9oYv2PXPk6ICae3OJc3Ey7CYATIE2idAI0dOhQHD9+HADw3nvvYeHChfD29sbYsWMxYcKEGgfy119/ITMzE+PHjwdQ1tJkZGQEa2trhf0cHByQmppa5XFCQkKwZcsWHD9+HCtXrsTp06fRv39/SKVVz5YaGhqKrKws4ZaQkFDj62CM1X+xGbG4++Qu9CX66OfVT+xwakwYCcZdYKK6knQFMpLBzcoNzhbOYofDqknpdt8VK1YI/x85cqTQIuPt7Y3Bg2v+Ternn39G//794excu1+eUaNGCf/39/dHq1at4OXlhVOnTqFXr16VPsfY2LhWrVeMMc0i7/7q6t4VNqY2IkdTc60dWkMCCZJykvAk7wnsGti9+ElM5bj7SzPVeh6g4OBgzJkzp1bJT1xcHI4dO4ZJkyYJ2xwdHVFcXIzMzEyFfdPS0pSq72nSpAkaNWqEhw8f1jg+xph20bTFT6tiYWwhzF/E3WDi4RXgNVONKv+Sk5Nx7tw5PH78GDKZTOGxGTNmKH28TZs2wd7eHgMHDhS2tW3bFoaGhjh+/DiGDx8OAIiMjER8fDyCg6u/ym5iYiLS09Ph5OSkdFyMMe2TXZSNU49OAdDM4e//FegUiKhnUQhLCUNfr75ih6NzZCTDxcSyulRuAdIsSidAmzdvxttvvw0jIyPY2toqTL4lkUiUToBkMhk2bdqEcePGwcDg33CsrKwwceJEzJkzBw0bNoSlpSXee+89BAcHK4wA8/X1xfLlyzF06FDk5uZi6dKlGD58OBwdHREdHY0PPvgATZs2Rb9+mtvPzxhTnSPRR1AiK0Ez22ZoZttM7HBqLdAxENvvbucWIJHcf3ofmYWZMDM0QyuHVmKHw5SgdAK0cOFCLFq0CKGhodDTq/1KGseOHUN8fHylBdRff/019PT0MHz4cBQVFaFfv374/vvvFfaJjIwUhuLr6+vj1q1b+OWXX5CZmQlnZ2f07dsXn3zyCdf4MMYAaE/3lxyPBBOXvPurfeP2MNQ3FDkapgylE6D8/HyMGjVKJckPAPTt2xdEVOljJiYm+O677/Ddd99V+fzyzzU1NcXhw4dVEhdjTPtIZVIciDoAQDu6v4B/R4I9SH+AnKIcWBhbiByRbuH6H82ldBYzceJE7NixQx2xMMaYWl1JuoKn+U9hbWKNzq6dxQ5HJewb2KOxRWMAEOaiYXWH6380l9ItQMuXL8egQYNw6NAh+Pv7w9BQscnvq6++UllwjDGmSvLh7yFNQ7SquyLQKRBJOUkISwlDF7cuYoejM9Lz03H/6X0AQEeXqlcnYPVTjRKgw4cPw8fHBwAqFEEzxlh9pcmrvz9PoGMg9j/Yz3VAdexSYtn6l76NfGFrZityNExZSidAq1atwsaNG4UZmxljTBM8ynyEO4/vQF+ij5CmIWKHo1JcCC0Orv/RbErXABkbG6NzZ+3oO2eM6Q756K/Obp3R0LShyNGolrwQ+u7juyiWFoscje64kMgzQGsypROgmTNn4ptvvlFHLIwxpjba2v0FAO5W7rAxsUGJrAR3H98VOxydUCItwZWkKwCAYNfqT87L6g+lu8CuXLmCEydOYP/+/WjRokWFIujdu3erLDjGGFOFnKIcYfbnQc20Y/6f8iQSCQIcA3Dy0UmEpYYJLUJMfW6l3UJ+ST6sTazh28hX7HBYDSidAFlbW2PYsGHqiIUxxtTiaMxRFEuL0bRhU/jY+ogdjloEOgaWJUApYQDnP2onr/8JdgmGnkQ18+KxuqVUAlRaWooePXqgb9++Si1IyhhjYpLX/wxuNlhrR6vKW324ELpucP2P5lMqbTUwMMDUqVNRVFSkrngYY0ylZCQTZn/Wxu4vOflIsJtpNyEj2Qv2ZrUljADjBEhjKd1u1759e4SF8TcMxphmuJJ0BY/zHsPS2BJd3bqKHY7a+DTygYmBCXKLc/Hw2UOxw9FqidmJiM+Kh55ED+0btxc7HFZDStcAvfvuu5g7dy4SExPRtm1bNGjQQOHxVq14NVzGWP0h7/7Sttmf/8tAzwCtHVrjctJl3Ei5oRUr3ddXFxPKlr9o7dAa5kbmIkfDakrpBGjUqFEAgBkzZgjbJBIJiAgSiQRSqVR10THGWC1p8/D3/wp0DMTlpMsISwnDqJajxA5Ha/H6X9pB6QQoNjZWHXEwxpjKxWXG4VbaLehJ9NC/aX+xw1E7LoSuG1z/ox2UToDc3d3VEQdjjKmcvPi5k2snnVirqfySGPJWeaZaBSUFuJFyAwAnQJpO6QQIAKKjo7F69WpEREQAAPz8/DBz5kx4eXmpNDjGGKsNXer+AgB/B3/oS/TxNP8pknKS4GLpInZIWud6ynWUyErgZO4EdytuENBkSo8CO3z4MPz8/HDlyhW0atUKrVq1wuXLl9GiRQscPXpUHTEyxpjScotzcSL2BADdSYBMDEzQ3K45AJRNiMhUTpgA0TWYW9g0nNItQPPnz8fs2bOxYsWKCts//PBD9OnTR2XBMcZYTR2LOYZiaTGa2DTRqaUKAh0DcefxHYSlhmGwj24kfnWJV4DXHkq3AEVERGDixIkVtk+YMAH37t1TSVCMMVZb+yL/7f7SpW/q5euAmGoRERdAaxGlEyA7OzuEh4dX2B4eHg57e3tVxMQYY7WiK7M/V0YYCcZdYCoXnRGNJ/lPYKRvhDZObcQOh9WS0l1gkydPxpQpUxATE4NOncoy4PPnz2PlypWYM2eOygNkjDFlXUu+hrS8NFgaW+Il95fEDqdOBTgGAADisuLwrOAZGpo2FDcgLSJv/QlyDoKxgbHI0bDaUjoBWrhwISwsLLBq1SqEhoYCAJydnbFkyRKFyREZY0ws8u6vfl79YKRvJHI0dcvaxBqe1p6IzYxFeGo4enr2FDskrcH1P9pF6S4wiUSC2bNnIzExEVlZWcjKykJiYiJmzpypU/3sjLH6a39U2fIXutb9JcfdYOrB9T/aRekEqDwLCwtYWFioKhbGGKu1hKwEhKeGQ0+ihwHeA8QORxRtHMvqU7gQWnWyCrNw5/EdAGVD4JnmUzoBSktLw5tvvglnZ2cYGBhAX19f4cYYY2KSL34a7BKMRmaNRI5GHPIWIPmMxaz2riRdAYHQxKYJHM0dxQ6HqYDSNUDjx49HfHw8Fi5cCCcnJ+72YozVK7re/QX8OxQ+Mj0S+SX5MDM0EzkizcfdX9pH6QTo3LlzOHv2LAICAtQQDmOM1VxecR6OxxwHoDuzP1fGycIJDg0ckJaXhltpt9DRpaPYIWm8C4lcAK1tlO4Cc3V1BRGpIxbGGKuV47HHUSQtgoe1B/zs/MQOR1RcCK06UpkUlxIvAeD6H22idAK0evVqzJ8/H48ePVJDOIwxVnO6OvtzZXhGaNW59+QesouyYW5kjpb2LcUOh6mI0l1gI0eORH5+Pry8vGBmZgZDQ0OFx589e6ay4BhjrLpkJBPqf3S5+0uOEyDVkdf/dGjcAQZ6Sn9ssnpK6Z/k6tWr1RAGY4zVzo2UG0jNTYW5kbnOzf5cGXkX2O202yiRlsBQ3/AFz2BVEep/uABaqyidAI0bN04dcTDGWK2Un/2ZlykAmtg0gYWRBXKKc3D/6X34O/iLHZLG4hFg2qlaNUB5eXlKHbS6+3t4eEAikVS4TZs2DQBQWFiIadOmwdbWFubm5hg+fDjS0tKee0wiwqJFi+Dk5ARTU1P07t0bUVFRSsXPGNM8+x78W//DAD2JnrAuGHeD1dzjvMd4+OwhAPBoOi1TrQSoadOmWLFiBVJSUqrch4hw9OhR9O/fH2vXrq3Wya9evYqUlBThdvToUQDAiBEjAACzZ8/Gvn37sGPHDpw+fRrJyckYNmzYc4/5+eefY+3atVi/fj0uX76MBg0aoF+/figsLKxWTIwxzZOUnYSw1DBIIEF/7/5ih1NvCHVAPBKsxi4mXAQAtLBrAWsTa3GDYSpVrS6wU6dOYcGCBViyZAlat26NoKAgODs7w8TEBBkZGbh37x4uXrwIAwMDhIaG4u23367Wye3s7BTur1ixAl5eXujWrRuysrLw888/4/fff0fPnmWL+W3atAnNmzfHpUuX0LFjxUyciLB69Wr873//wyuvvAIA2LJlCxwcHPDXX39h1KhR1YqLMaZZ5LM/d3TpCPsG9iJHU38IQ+G5BajGuPtLe1UrAfLx8cGuXbsQHx+PHTt24OzZs7hw4QIKCgrQqFEjBAYGYsOGDejfv3+Nl8MoLi7G1q1bMWfOHEgkEly/fh0lJSXo3bu3sI+vry/c3Nxw8eLFShOg2NhYpKamKjzHysoKHTp0wMWLF6tMgIqKilBUVCTcz87OrtE1MMbEwbM/V07eAhSeGg4i0vmpAWqCC6C1l1JF0G5ubpg7dy7mzp2r8kD++usvZGZmYvz48QCA1NRUGBkZwdraWmE/BwcHpKamVnoM+XYHB4dqPwcAli9fjqVLl9Y8eMaYaPJL8nEs5hgArv/5Lz87PxjpGyGrKAuxmbFoYtNE7JA0SrG0GNeSrwHgBEgb1Wo1eFX6+eef0b9/fzg7O9f5uUNDQ5GVlSXcEhIS6jwGXSCVSXHq0Slsu70Npx6dglQmFTskpgWOxxxHYWkh3K3ceZK6/zDUN4S/fdnoL14YVXnhqeEoLC2EraktvBt6ix0OU7F6MaNTXFwcjh07ht27dwvbHB0dUVxcjMzMTIVWoLS0NDg6Vr4Sr3x7WloanJycFJ7zvLXLjI2NYWzMw2bVaXfEbsw8NBOJ2YnCNhdLF6wJWYNhzZ9f2M7Y88jrfwY1G8RdPJUIdAzE9ZTrCEsJw6t+r4odjkaR1/8Euwbz75YWqhctQJs2bYK9vT0GDhwobGvbti0MDQ1x/PhxYVtkZCTi4+MRHFz5Wiyenp5wdHRUeE52djYuX75c5XOY+u2O2I1Xt7+qkPwAZSN3Xt3+KnZH7K7imYw9HxHx7M8vwIXQNScUQPMCqFpJ9ARIJpNh06ZNGDduHAwM/m2QsrKywsSJEzFnzhycPHkS169fx1tvvYXg4GCFAmhfX1/s2bMHACCRSDBr1ix8+umn2Lt3L27fvo2xY8fC2dkZQ4YMqetLYyjr9pp5aCYIFRfQlW+bdWgWd4exGrmRcgPJOcloYNgA3T26ix1OvcRLYtQMEeF8wnkAXP+jrUTvAjt27Bji4+MxYcKECo99/fXX0NPTw/Dhw1FUVIR+/frh+++/V9gnMjISWVlZwv0PPvgAeXl5mDJlCjIzM9GlSxccOnQIJiYmar8WVtFf9/+q0PJTHoGQkJ2Axl81hldDLzS2aAxnC+d//7X8934DowZ1GDnTBPLur75efXn25yq0cmgFCSRIzU1Fam4qHM0rLyFgihKyE5Cckwx9iT7aNW4ndjhMDSREVPGr+XMcOnQI5ubm6NKlCwDgu+++w4YNG+Dn54fvvvsONjY2agm0LmVnZ8PKygpZWVmwtLQUOxyNQkQITw3Hvgf7sO/BPmEEhSpYGVtVSIr+myw5mjvyYoU6JOjHIFxPuY6NL2/EW4FviR1OvdX8u+a4//Q+/nnjH54ospr+uPMHXt/1OoKcg3B18lWxw2HVpMznt9KfFPPmzcPKlSsBALdv38bcuXOFbqo5c+Zg06ZNNYuaaayCkgKciD2BfQ/2Yf+D/UjKSVL6GN/2/xYO5g5IzklGUnYSknP//9+cZCTlJCG3OBdZRVnIKspCxNOIKo8jgQQO5g6VJkfl/29jYiNKUaNUJsXZ+LNIyUmBk4UTurp1hb5ezebO0nXJOcm4nnIdEkgwwHuA2OHUa4GOgbj/9D7CUsM4Aaomrv/RfkonQLGxsfDz8wMA7Nq1C4MGDcKyZctw48YNDBjAb0K6IiUnBQeiDmDfg304FnMM+SX5wmNmhmbo06QPBjUbhBCvEARvDEZSdlKldUASSOBi6YKpQVOfmwhkF2X/mxz9f1Ik/1e+LSU3BaWyUqGp/3rK9SqPZ2JgAmcL5wotSf9tXTI1NK3dC1UOj4RTrQMPDgAA2jduDwdzhxfsrdsCHQOx7c42rgNSAs8Arf2UToCMjIyQn1/2YXfs2DGMHTsWANCwYUOeQVmLvahry8XSBYO8B2Gwz2D08OihkDisCVmDV7e/CgkkCkmQBGUtMKtDVr+wFcTS2BKWxpbwbeRb5T4ykuFJ3pN/k6NyyVL5bekF6SgsLURMRgxiMmKee14bE5sK3W7/rU9yaODwwvjlI+H+mwTKR8LtfG0nJ0FK4sVPq08YCcZrglVLXnEewlPDAXACpM2UToC6dOmCOXPmoHPnzrhy5Qr+/PNPAMCDBw/g4uKi8gCZeORdW/sf7Mf+qP0VipnbObfD4GaDMajZIAQ4BlTZpTSs+TDsfG1npa0fq0NWq+yDX0+iBwdzBziYO6CNU5sq9yssLURKTopiS1J2UoVWpYLSAmQUZiCjMAN3n9x97nkdzR3/bUEyV0yYHBo44L2D71U5Ek4CCWYdmoVXfF7h7rBqKigpEGZ/5uUvXkw+Eiw6IxpZhVmwMrESOaL67WryVUhJChdLF7hauYodDlMTpROgb7/9Fu+++y527tyJdevWoXHjxgCAgwcPIiQkROUBsrr1vK4tUwNT9PHqg8HNBmOg90A4WTg950iKhjUfhld8XqkX9S8mBibwtPGEp41nlfsQEbKKsiq2JP2nPik1NxVSkiI5JxnJOcm4mqx8saR8JNzZ+LM8lLuaTsSeQEFpAVwtXdHKoZXY4dR7tma2cLV0RUJ2Am6m3cRL7i+JHVK9Jl8Bnlt/tJvSCZCbmxv2799fYfvXX3+tkoBY3apN15ay9PX0NeYDXiKRwNrEGtYm1vCz86tyP6lMisd5j6tsSUrOSUZsRizySvJeeM6UnBRVXoJWK9/9xTP0Vk8bpzZIyE5AWEoYJ0AvICyAygXQWq1G44Wjo6OxadMmREdHY82aNbC3t8fBgwfh5uaGFi1aqDpGpmIFJQU4+egk9kXuq1XXFitL6pwsnJ7bGnbq0Sn0+KXHC4+lTIuaLiMiheUvWPUEOgbi78i/uRD6BYiIC6B1hNIJ0OnTp9G/f3907twZZ86cwWeffQZ7e3vcvHkTP//8M3bu3KmOOFktqatri71YV7eucLF0qXIkHAC4Wrqiq1vXOo5MM4WnhiMpJwlmhmbo4fnixJKVkRdC86Koz/cg/QGeFTyDiYEJWju2FjscpkZKJ0Dz58/Hp59+ijlz5sDCwkLY3rNnT3z77bcqDY7VnLxra/+D/dj3YF+F2pTGFo0xuNlglXRtsefT19OvciScXEv7ltzSVk3y1p8+TfrAxIBneK8ueSH0vSf3UFhayK9dFeStP+2c28FI30jkaJg6KZ0A3b59G7///nuF7fb29nj69KlKgmI1U1haWDYhIXdt1TtVjYSzNrFGZmEmDj48iDG7x2DzkM38pvsCPPy9ZlwsXWBraov0gnTceXwHQc5BYodUL3H3l+5QOgGytrZGSkoKPD0VR9CEhYUJI8JY3UnNTS0bpv5gP47GHOWurXqsqpFw2+9ux9i/xmLbnW1IL0jHrtd2wdzIXOxw66WUnBShNXNgs4EiR6NZJBIJAp0CcSzmGMJSwjgBqoJQAM0JkNZTOgEaNWoUPvzwQ+zYsQMSiQQymQznz5/H+++/L0yKyNSHiHAz7Sb2Re7jri0NVNlIuNf9X0dD04YYvn04jkQfQa8tvXDgjQNoZNZInCDrsX+i/gFQ1prJi3oqL9Dx/xMgLoSuVEZBBu49uQcACHYJFjkapm5KJ0DLli3DtGnT4OrqCqlUCj8/P0ilUrzxxhv43//+p44YdV51urYGNRuEwc0Gc9eWhurXtB+Ojz2Ogb8PxJWkK+iysQuOvHkEblZuYodWr3D3V+3I64A4AarcpcRLAADvht6wa2AncjRM3Wq0FMaGDRuwaNEi3L59G7m5uQgMDIS3t7c64tMqyiyEmZqbigMPykZtcdeWbujg0gFn3zqLflv7ITI9Ep03dsbhMYefOw+RLiksLcTRmKMAgME+nADVhHwk2K20W5DKpDzz+H9w/Y9uUToB+vjjj/H+++/D1dUVrq7/ThFeUFCAL774AosWLVJpgNriRQthctcWA4Dmds1xfsJ59NvaDxFPI9B1U1cceOMAOrp0FDs00Z2MPYn8kny4WLqgtQMPT64J74beMDM0Q35JPh6kP0Bzu+Zih1SvcP2PbpEQUeUTk1RBX18fKSkpsLe3V9ienp4Oe3t7SKVSlQYohuzsbFhZWSErKwuWlpa1Pl5VC2HKh0T38+qHe0/uISE7QeFx7trSXen56Rj4+0BcTroMM0Mz7HptF0Ka6vZSM+8eeBfrrq3D1LZTsW7QOrHD0Vidfu6Ei4kX8duw3/CG/xtih1NvlMpKYb3CGnklebj9zm20tG8pdkisBpT5/NZT9uBEVOkH8c2bN9GwYUNlD6f1pDIpZh6aWeVCmABwOPowErITYGpgipd9XsaGwRuQPCcZVyZfwaJuixDoFMjJj46xNbPF8bHHEdI0BPkl+Ri8bTB+u/Wb2GGJhmd/Vh2hDohXhldw5/Ed5JXkwdLYkruddUS1u8BsbGwgkUggkUjQrFkzhQ9kqVSK3NxcTJ06VS1BarKz8WcrFC1XZlnPZZjVcRZ3bTFBA6MG2DtqL976+y38dvs3jNkzBk/zn2Jmx5lih1bnbqXdEr4k9PTsKXY4Gk1eB8SF0Irk9T/BLsHQkyjdNsA0ULUToNWrV4OIMGHCBCxduhRWVlbCY0ZGRvDw8EBwMA8b/K/qLnDpYe3ByQ+rwFDfEFuGbkEjs0ZYc3kNZh2ehcd5j/Fpz091qlVQPvqrj1cf/juppTZObQCUJUBVtejrovIJENMN1U6Axo0bBwDw9PREp06dYGhoqLagtEl1R2jxSC5WFT2JHr7u9zUcGjhgwYkFWHZuGR7nPca6QetgoFej9Yw1jtD95c3dX7XVwq4FDPQM8KzgGeKz4uFu7S52SPUCjwDTPUq/e3br1g1SqRS7du1CREQEAKBFixZ4+eWXoa/PQyr/60ULYUoggYulCy+EyZ5LIpEgtGso7BrY4e39b+OnsJ/wtOAptg3fpvVrOqXlpuFK0hUAXP+jCsYGxmhh1wI3024iLDWMEyCUtdTHZsZCAgk6uHQQOxxWR5Tu6Hz48CGaN2+OsWPHYvfu3di9ezfGjBmDFi1aIDo6Wh0xajT5QphAWbJTnvz+6pDVPB8Hq5ZJbSZh12u7YKxvjL/u/4WQrSHIKswSOyy1OhB1AARCkHMQt5SqiFAHxIXQAICLiRcBAP4O/rA0rv3IX6YZlE6AZsyYAS8vLyQkJODGjRu4ceMG4uPj4enpiRkzZqgjRo0nXwizsaXiWmkuli7Y+dpODGs+TKTImCYa4jsEh8cchqWxJU7HnUa3zd2Qmpsqdlhqw91fqsczQisSur9cuPtLlyjdBXb69GlcunRJYci7ra0tVqxYgc6dO6s0OG1S1UKY3PLDaqKbRzecHn8aIVtDcDPtJjpv7IwjY47Aq6GX2KGpVGFpIY5EHwHAsz+rEidAirj+Rzcp3QJkbGyMnJycCttzc3NhZGSkkqC0lXwhzNf9X0d3j+6c/LBaCXAMwPkJ59HEpgliMmLQeWNnhKeGix2WSp16dAp5JXlwtnAWPrRZ7bV2LJtJOzE7EU/zn4ocjbgKSwtxPeU6AE6AdI3SCdCgQYMwZcoUXL58GUQEIsKlS5cwdepUvPzyy+qIkTFWBa+GXjg/4TxaO7RGWl4aum3uhtOPTosdlsqU7/7i4dqqY2lsiaYNmwLgOqAbKTdQLC2GfQN7NLFpInY4rA4pnQCtXbsWXl5eCA4OhomJCUxMTNC5c2c0bdoUa9asUUeMjLHncDR3xOnxp/GS+0vILspGv6398Nf9v8QOq9aI6N/V37n7S+W4G6xM+e4vTrJ1i9IJkLW1Nf7++29ERkZi586d2LlzJyIjI7Fnzx6FyREZY3XHysQKh8ccxhDfISiSFmH49uH4+cbPYodVK3ce30F8VjxMDEx49mc14ASoDBdA6y6lE6Bz584BALy9vTF48GAMHjwYTZs2VXlgjDHlmBiYYMeIHZgYOBEykmHSvklYcW4FlFzvuN6Qt/70btIbZoZmIkejfXgofFkro3wIPNf/6B6lE6CePXvC09MTCxYswL1799QRE2Oshgz0DLBh8AaEdgkFAIQeD8Wcw3MgI5nIkSlP6P5qxt1f6iBvAXqQ/gC5xbkiRyOOR5mPkJqbCkM9Q7R1bit2OKyOKZ0AJScnY+7cuTh9+jRatmyJgIAAfPHFF0hMfPGCn4wx9ZNIJFjWaxm+7vc1AGD15dUYu2csSqQlIkdWfY/zHuNy4mUAwEDvgSJHo50czB3gbOEMAuFW2i2xwxGFvPurjVMbrZ9RnVWkdALUqFEjTJ8+HefPn0d0dDRGjBiBX375BR4eHujZk/vpGasvZnWcha1Dt8JAzwC/3f4Nr/zxCvKK88QOq1r+ifoHBEIbpzYVJhBlqiPUAeloNxjP/6PblE6AyvP09MT8+fOxYsUK+Pv74/Rp5YffJiUlYcyYMbC1tYWpqSn8/f1x7do14fG0tDSMHz8ezs7OMDMzQ0hICKKiop57zM2bN0MikSjcTEw4u2e6Z3Sr0dg7ai9MDUxx8OFB9P61N54VPBM7rBeSd3/x7M/qJU+AbqTcEDkScVxI5ARIl9U4ATp//jzeffddODk54Y033kDLli1x4MABpY6RkZGBzp07w9DQEAcPHsS9e/ewatUq2NjYACgrUBsyZAhiYmLw999/IywsDO7u7ujduzfy8p7/TdbS0hIpKSnCLS4urqaXyphG6+/dH8fHHoeNiQ0uJV5C101dkZhdf7usi0qLePbnOiIUQuvgSLCcohyh648TIN2k9FIYoaGh+OOPP5CcnIw+ffpgzZo1eOWVV2BmpvwojZUrV8LV1RWbNm0Stnl6egr/j4qKwqVLl3Dnzh20aNECALBu3To4Ojpi27ZtmDRpUpXHlkgkcHR0VDomxrRRsGswzk04h35b++Hek3vo9HMnHHnzCHwb+YodWgWn404jtzgXTuZOaOPURuxwtJq8BejO4zsolhbDSF93ZvO/knQFMpLB3codzhbOYofDRKB0C9CZM2cwb948JCUlYf/+/Xj99ddrlPwAwN69exEUFIQRI0bA3t4egYGB2LBhg/B4UVERACh0X+np6cHY2FgYjl+V3NxcuLu7w9XVFa+88gru3r1b5b5FRUXIzs5WuDGmbfzs/HB+wnn42PogITsBXTZ2wZWkK2KHVYF89ueB3gOhJ6lVLz17AQ9rD1ibWKNEVoJ7T3RrVC/X/zCl313kXV+NGjWq9cljYmKwbt06eHt74/Dhw3jnnXcwY8YM/PLLLwAAX19fuLm5ITQ0FBkZGSguLsbKlSuRmJiIlJSUKo/r4+ODjRs34u+//8bWrVshk8nQqVOnKkeqLV++HFZWVsLN1dW11tfGWH3kZuWGcxPOoZ1zO6QXpKPnLz2F7qb6gGd/rlsSiQQBjgEAdK8Qmut/mIRqMEtaZGQkvvnmG0RERAAAmjdvjvfeew8+Pj5KHcfIyAhBQUG4cOGCsG3GjBm4evUqLl4sm5zq+vXrmDhxIm7evAl9fX307t0benp6ICIcPHiwWucpKSlB8+bN8frrr+OTTz6p8HhRUZHQ2gQA2dnZcHV1RVZWFiwtLZW6JsY0QW5xLob9OQxHY47CUM8Qvwz5Ba/7vy52WLjz+A781/nDxMAE6R+k8wSIdWDO4Tn4+tLXeK/9e1jbf63Y4dQJGcnQcGVDZBVl4fqU69zVqkWys7NhZWVVrc9vpVuAdu3ahZYtW+L69eto3bo1WrdujRs3bqBly5bYtWuXUsdycnKCn5+fwrbmzZsjPj5euN+2bVuEh4cjMzMTKSkpOHToENLT09GkSfUXrTM0NERgYCAePnxY6ePGxsawtLRUuDGmzcyNzLH/jf0Y1XIUSmQlGL17NL65/I3YYQndXz09e3LyU0d0cUmMiCcRyCrKgpmhGVo5tBI7HCYSpYugP/jgA4SGhuLjjz9W2L548WJ88MEHGD58eLWP1blzZ0RGRipse/DgAdzd3SvsK19nLCoqCteuXau0JacqUqkUt2/fxoABA6r9HMa0nZG+EX4b9hsamTbCt1e/xYxDM/Ak/wmWdl8q2qKQPPtz3ZOPBAtPDYeMZDpRdyWv/+nQuAMM9JT+GGRaQunf9JSUFIwdO7bC9jFjxjy3Lqcys2fPxqVLl7Bs2TI8fPgQv//+O3788UdMmzZN2GfHjh04deqUMBS+T58+GDJkCPr27SvsM3bsWISGhgr3P/74Yxw5cgQxMTG4ceMGxowZg7i4uOeOGmNMF+lJ9LC2/1p80qPsC8UnZz7BOwfegVQmrfNYnuQ9wcWEsq5vnv257vg28oWJgQlyi3MR/Sxa7HDqBK//xYAaJEDdu3fH2bNnK2w/d+4cunbtqtSx2rVrhz179mDbtm1o2bIlPvnkE6xevRqjR48W9klJScGbb74JX19fzJgxA2+++Sa2bdumcJz4+HiF5CsjIwOTJ09G8+bNMWDAAGRnZ+PChQsVutsYY2WFsP976X9YP3A9JJDgh+s/YOTOkSgsLazTOA4+PAgCIcAxAK5WPBChrhjoGcDf3h+A7nSDyVuAgl2CRY6EiUnpIuj169dj0aJFeO2119CxY0cAwKVLl7Bjxw4sXboUzs7/zqfw8ssvqzbaOqJMERVj2mTXvV14Y/cbKJYWo4dHD/w16i9YGtfN38CIHSOw895OLHxpIT7u8fGLn8BUZur+qfjh+g+Y33k+lvdeLnY4avU0/ynsvrAr+/+8p7A1sxU5IqZKynx+K50A6elVr9FIIpFAKq37ZnRV4ASI6bKTsSfxyh+vIKc4B4GOgTg4+iAczB3Ues5iaTEafd4IOcU5uDzpMto3bq/W8zFFP1z7AVMPTEU/r344NOaQ2OGo1f4H+zF422D4NvJFxLQIscNhKqbWUWAymaxaN01NfhjTdT08e+DU+FOwM7NDWGoYumzqgtiMWLWe80zcGeQU58DR3BFBzkFqPRerSF4IfSPlBmowM4pGESZAdOH6H12n/eX+jDGltXFqg/MTzsPD2gMPnz1Ep42dhHWT1GFfZNnoL579WRz+9v7Ql+jjSf4TJOckix2OWvEM0ExO6fF//x3+/l+LFi2qcTCMsfrD29Yb5yecR8jWENx+fBsvbXoJ+17fh67uyg12eJHysz8Pasarv4vB1NAUvo18cffJXYSlhqGxZWOxQ1KLEmmJsPwLJ0BM6QRoz549CvdLSkoQGxsLAwMDeHl5cQLEmBZxtnDGmbfO4OVtL+Ns/Fn03doXf776J172Ud0Ah4inEYjNjIWxvjF6N+mtsuMy5QQ6BZYlQClhWpuI3ky7iYLSAtiY2MCnkXIrFzDto3Rbc1hYmMLtzp07SElJQa9evTB79mx1xMgYE5G1iTUOjzmMwc0Go7C0EMP+HIZNYZtUdnx591cPzx4wNzJX2XGZcnRhRmhh+LtrMHe1MtXUAFlaWmLp0qVYuHChKg7HGKtnTA1NsXvkbowPGA8pSTFh7wR8fv5zlRx7f1TZ8hc8+7O4dCkB4gJoBqiwCDorKwtZWVmqOhxjrJ4x0DPAxpc34oNOHwAAPjz2Id4/8j5kJKvxMdPz04UPJW3tdtEU8lXhH2U+QkZBhrjBqAkXQLPylK4BWrtWcbVgIkJKSgp+/fVX9O/fX2WBMcbqH4lEgpV9VsK+gT3eP/o+Vl1chSf5T/DT4J9gqG+o9PEOPjwIGcnQyqEV3Kzc1BAxqy4bUxt4WHvgUeYjhKeGo4dnD7FDUqnE7EQkZCdAX6KPdo3biR0OqweUToC+/vprhft6enqws7PDuHHjFNbjYoxpr7md5sKugR0m/D0BW25uQXp+OraP2K70Cu68+Gn9EugYiEeZjxCWGqZ1CZB8nblWDq241owBqEECFBur3gnRGGOaYWzrsbA1tcWIHSNwIOoA+vzaB/tf3w8bU5tqPb9YWoxDD8tmHeYEqH4IdAzEnvt7tLIOiLu/2H9xGTxjrMYGNhuIo28ehbWJNS4kXEDXTV2RlJ1Ureeeiz+H7KJs2Dew5y6JekI+I3RYihYmQImcADFFnAAxxmqls1tnnH3rLJwtnHH3yV102tgJkU8jX/g8nv25/mnj1AYAcP/pfRSUFIgcjeoUlBTgRsoNAJwAsX/xuw5jrNZa2rfEhQkX0My2GeKz4tFlUxdcS75W5f48+3P95GTuBPsG9pCSFLcf3xY7HJW5lnwNpbJSOJk7wd3KXexwWD3BCRBjTCXcrd1x7q1zCHIOwtP8p+jxSw8cizlW6b6R6ZGIzoiGkb4R+nr1reNIWVUkEokwH5C8xUQblK//kUgkIkfD6otqJUBt2rRBRkbZvBAff/wx8vPz1RoUY0wz2TWww4mxJ9DLsxdyi3Mx4LcB2H53e4X9hNmfPXj25/pGmBBRi+qAuP6HVaZaCVBERATy8vIAAEuXLkVubq5ag2KMaS4LYwsceOMARviNQImsBKN2jsL3V78HAEhlUpx6dAobwzcCAAZ4DxAzVFYJoRBaS0aCERGPAGOVqtYw+ICAALz11lvo0qULiAhffvklzM0r/9bGi6EyxowNjLFt+DbYmdnh+2vfY9o/03D60WlcSLyAxOxEYb8V51bAxdIFw5oPEzFaVp68Bej249solZXCQE/p2VLqlYfPHuJp/lMY6xsL18YYUM0EaPPmzVi8eDH2798PiUSCgwcPwsCg4lMlEgknQIwxAIC+nj6+HfAt7BvYY8npJdh+r2JXWGpuKl7d/ip2vraTk6B6wquhFyyMLJBTnIP7T++jpX1LsUOqFXnrT5BzEIwNjEWOhtUn1UqAfHx88McffwAom/n5+PHjsLe3V2tgjDHNJ5FI8L+X/ofVl1cjszCzwuMEggQSzDo0C6/4vAJ9Pf26D5Ip0JPoobVja5yLP4ewlDCtSYC4+4v9l9KjwGQyGSc/jLFqOxt/ttLkR45ASMhOwNn4s3UXFHsubVoZ/mJi2RIYwS7BIkfC6psade5GR0dj9erViIiIAAD4+flh5syZ8PLyUmlwjDHNl5KTotL9mPppSwKUVZiFO4/vAACCXTkBYoqUbgE6fPgw/Pz8cOXKFbRq1QqtWrXC5cuX0aJFCxw9elQdMTLGNJiThZNK92PqJx8JFp4aDiISOZqau5x0GQRCE5smcDR3FDscVs8o3QI0f/58zJ49GytWrKiw/cMPP0SfPn1UFhxjTPN1desKF0sXJGUngVDxw1QCCVwsXdDVrasI0bHK+Nn5wVDPEJmFmXiU+QieNp5ih1QjXP/DnkfpFqCIiAhMnDixwvYJEybg3r17KgmKMaY99PX0sSZkDYCyZKc8+f3VIau5ALoeMdI3gr+DPwDN7gYTEiAXToBYRUonQHZ2dggPD6+wPTw8nIujGWOVGtZ8GHa+thONLRsrbHexdOEh8PWUps8ILZVJcSnxEgBuAWKVU7oLbPLkyZgyZQpiYmLQqVPZL9X58+excuVKzJkzR+UBMsa0w7Dmw/CKzys4G38WKTkpcLJwQle3rtzyU09peiH03Sd3kVOcA3Mjc40fys/UQ+kEaOHChbCwsMCqVasQGhoKAHB2dsaSJUswY8YMlQfIGNMe+nr66O7RXewwWDXIC6E1dVFUefdXR5eOnGSzSimdAEkkEsyePRuzZ89GTk4OAMDCwkLlgTHGGBNPK4dWkECClNwUpOWmwcHcQeyQlML1P+xFlK4BKs/CwoKTH8YY00LmRuZoZtsMgGZ2g/EIMPYitUqAGGOMaS9hZXgNK4ROy01DdEY0JJCgg0sHscNh9RQnQIwxxiqlqYXQ8uUvWti3gLWJtbjBsHqLEyDGGGOV0tgEKIHX/2IvplQCVFJSgl69eiEqKkplASQlJWHMmDGwtbWFqakp/P39ce3aNeHxtLQ0jB8/Hs7OzjAzM0NISEi1zr9jxw74+vrCxMQE/v7++Oeff1QWM2OM6QJ5F9jDZw+RXZQtcjTVdyGR63/YiymVABkaGuLWrVsqO3lGRgY6d+4MQ0NDHDx4EPfu3cOqVatgY2MDACAiDBkyBDExMfj7778RFhYGd3d39O7dG3l5eVUe98KFC3j99dcxceJEhIWFYciQIRgyZAju3LmjstgZY0zbNTJrBBdLFwDAzdSbIkdTPcXSYlxNugqAEyD2fBJScqW72bNnw9jYuMJaYDUxf/58nD9/HmfPnq308QcPHsDHxwd37txBixYtAAAymQyOjo5YtmwZJk2aVOnzRo4ciby8POzfv1/Y1rFjRwQEBGD9+vUV9i8qKkJRUZFwPzs7G66ursjKyoKlpWVtLpExxjTay9texr4H+7AmZA1mdKj/c71dTryMjj93hK2pLZ7MewKJRPLiJzGtkZ2dDSsrq2p9fitdA1RaWop169YhKCgIb7/9NubMmaNwU8bevXsRFBSEESNGwN7eHoGBgdiwYYPwuDwpMTEx+TdgPT0YGxvj3LlzVR734sWL6N27t8K2fv364eLFi5Xuv3z5clhZWQk3V1dXpa6DMca0labVAZUf/s7JD3sepROgO3fuoE2bNrCwsMCDBw8QFhYm3CpbI+x5YmJisG7dOnh7e+Pw4cN45513MGPGDPzyyy8AAF9fX7i5uSE0NBQZGRkoLi7GypUrkZiYiJSUlCqPm5qaCgcHxUm7HBwckJqaWun+oaGhyMrKEm4JCQlKXQdjjGmrNk5tAGjOUHiu/2HVpfRM0CdPnlTZyWUyGYKCgrBs2TIAQGBgIO7cuYP169dj3LhxMDQ0xO7duzFx4kQ0bNgQ+vr66N27N/r37w8le+6ey9jYGMbGxio7HmOMaQt5IfTdJ3dRVFoEY4P6+15JRDwBIqu2Gg+Df/jwIQ4fPoyCggIAqFFC4uTkBD8/P4VtzZs3R3x8vHC/bdu2CA8PR2ZmJlJSUnDo0CGkp6ejSZMmVR7X0dERaWlpCtvS0tLg6OiodIyMMabLXC1d0dC0IUplpbj75K7Y4TxXfFY8knOSYaBngCDnILHDYfWc0glQeno6evXqhWbNmmHAgAFCV9TEiRMxd+5cpY7VuXNnREZGKmx78OAB3N3dK+xrZWUFOzs7REVF4dq1a3jllVeqPG5wcDCOHz+usO3o0aMIDuY5IRhjTBkSiUSoA6rvC6PKW38CHQNhZmgmcjSsvlM6AZo9ezYMDQ0RHx8PM7N/f8FGjhyJQ4cOKX2sS5cuYdmyZXj48CF+//13/Pjjj5g2bZqwz44dO3Dq1ClhKHyfPn0wZMgQ9O3bV9hn7Nixwsr0ADBz5kwcOnQIq1atwv3797FkyRJcu3YN06dPV/ZyGWNM5wmF0PW8Doi7v5gylK4BOnLkCA4fPgwXFxeF7d7e3oiLi1PqWO3atcOePXsQGhqKjz/+GJ6enli9ejVGjx4t7JOSkoI5c+YgLS0NTk5OGDt2LBYuXKhwnPj4eOjp/ZvLderUCb///jv+97//YcGCBfD29sZff/2Fli1bKnu5jDGm84Q1wer5SDAugGbKUHoeIAsLC9y4cQPe3t6wsLDAzZs30aRJE1y7dg39+vVDenq6umKtM8rMI8AYY9ou4kkE/L73g5mhGbLnZ0NfT1/skCrIK86D1QorSEmK+FnxcLXi6Ux0kVrnAeratSu2bNki3JdIJJDJZPj888/Ro0cP5aNljDFWrzWzbQYzQzPkl+Qj6pnqlkJSpavJVyElKVwsXTj5YdWidBfY559/jl69euHatWsoLi7GBx98gLt37+LZs2c4f/68OmJkjDEmIn09fbRyaIVLiZcQlhIG30a+YodUAdf/MGUp3QLUsmVLPHjwAF26dMErr7yCvLw8DBs2DGFhYfDy8lJHjIwxxkRW32eEFhIgF06AWPUo3QIElA1J/+ijj1QdC2OMsXqqPidAMpLhYmLZUkfcAsSqq0YJUEZGBn7++WdEREQAAPz8/PDWW2+hYcOGKg2OMcZY/SCMBEsJAxHVq3W2HqQ/wLOCZzA1MEWAY4DY4TANoXQX2JkzZ+Dh4YG1a9ciIyMDGRkZWLt2LTw9PXHmzBl1xMgYY0xkLe1bwkDPAOkF6UjMThQ7HAXy7q92jdvBUN9Q5GiYplA6AZo2bRpGjhyJ2NhY7N69G7t370ZMTAxGjRqlMIEhY4wx7WFiYAI/u7Kli+pbNxjX/7CaUDoBevjwIebOnQt9/X/ngdDX18ecOXPw8OFDlQbHGGOs/qivM0LzCDBWE0onQG3atBFqf8qLiIhA69atVRIUY4yx+qc+FkI/K3iGiKdln0nBrrzeI6u+ahVB37p1S/j/jBkzMHPmTDx8+BAdO3YEAFy6dAnfffcdVqxYoZ4oGWOMiU5eCF2fFkW9lHgJQNlkjY3MGokcDdMk1UqAAgICIJFIUH7VjA8++KDCfm+88QZGjhypuugYY4zVG/IRVgnZCUjPT4etma24AeHf7q9gF279YcqpVgIUGxur7jgYY4zVc5bGlvCy8UJ0RjTCUsPQu0lvsUPi+h9WY9VKgNzd3dUdB2OMMQ0Q6BRYlgCliJ8AlcpKcSXpCgBOgJjyajQRYnJyMs6dO4fHjx9DJpMpPDZjxgyVBMYYY6z+CXQMxM57O+tFIfTttNvIK8mDpbGlMESfsepSOgHavHkz3n77bRgZGcHW1lZhNlCJRMIJEGOMabH6NBKsfP2PnkTpQc1MxymdAC1cuBCLFi1CaGgo9PT4F44xxnSJfCRY5NNI5BXnoYFRA9FiuZDI9T+s5pTOYPLz8zFq1ChOfhhjTAc5mjvC0dwRBMKttFsvfoIacQE0qw2ls5iJEydix44d6oiFMcaYBmjj1AaAuN1gyTnJeJT5CHoSPbRv3F60OJjmUroLbPny5Rg0aBAOHToEf39/GBoqLjz31VdfqSw4xhhj9U+gYyD+ifpH1CUxLiZcBAD42/vD0thStDiY5qpRAnT48GH4+PgAQIUiaMYYY9qtPhRCc/cXqy2lE6BVq1Zh48aNGD9+vBrCYYwxVt/JC6FvP76NEmkJDPUNX/AM1eMCaFZbStcAGRsbo3PnzuqIhTHGmAbwtPaElbEViqXFuPfkXp2fv7C0ENeTrwPgJTBYzSmdAM2cORPffPONOmJhjDGmASQSibAumBjdYNeTr6NEVgL7BvZoYtOkzs/PtIPSXWBXrlzBiRMnsH//frRo0aJCEfTu3btVFhxjjLH6KdAxEKfjTiMsJQzjA8bX6bkvJpYVQHdy7cS1p6zGlE6ArK2tMWzYMHXEwhhjTEPI64DEaAESCqBduP6H1ZzSCdCmTZvUEQdjjDENIh8JFp4aDhnJ6mwpCiLiEWBMJXg6Z8YYY0rzbeQLY31j5BTnICYjps7OG5sZi7S8NBjqGaKtc9s6Oy/TPkq3AHl6ej63zzUmpu7+EBhjjInDUN8Q/g7+uJZ8DWEpYWjasGmdnFfe+tPWuS1MDEzq5JxMOymdAM2aNUvhfklJCcLCwnDo0CHMmzdPVXExxhir5wIdA8sSoNQwjGgxok7OyfU/TFWUToBmzpxZ6fbvvvsO165dq3VAjDHGNIMYM0Jz/Q9TFZXVAPXv3x+7du1S1eEYY4zVc8KiqHW0Jlh2UTZuP74NAAh25QkQWe2oLAHauXMnGjZsqKrDMcYYq+f8HfyhJ9FDWl4aUnJS1H6+K0lXICMZPKw94GzhrPbzMe2mdAIUGBiINm3aCLfAwEA4OTlhwYIFWLBggdIBJCUlYcyYMbC1tYWpqSn8/f0VutJyc3Mxffp0uLi4wNTUFH5+fli/fv1zj7l582ZIJBKFm4kJF8sxxpgqmRmawbeRL4C66Qbj7i+mSkrXAA0ZMkThvp6eHuzs7NC9e3f4+voqdayMjAx07twZPXr0wMGDB2FnZ4eoqCjY2NgI+8yZMwcnTpzA1q1b4eHhgSNHjuDdd9+Fs7MzXn755SqPbWlpicjISOE+zxbKGGOqF+gYiHtP7iEsJQwDvAeo9VzyBIjX/2KqoHQCtHjxYpWdfOXKlXB1dVWYXNHT01NhnwsXLmDcuHHo3r07AGDKlCn44YcfcOXKlecmQBKJBI6OjiqLlTHGWEWBjoH47fZvuJF6Q63nkZEMlxIvAeAWIKYaok6EuHfvXgQFBWHEiBGwt7dHYGAgNmzYoLBPp06dsHfvXiQlJYGIcPLkSTx48AB9+/Z97rFzc3Ph7u4OV1dXvPLKK7h7926V+xYVFSE7O1vhxhhj7MWEJTHUXAgd8SQCWUVZMDM0QyuHVmo9F9MN1U6A9PT0oK+v/9ybgYFyDUoxMTFYt24dvL29cfjwYbzzzjuYMWMGfvnlF2Gfb775Bn5+fnBxcYGRkRFCQkLw3Xff4aWXXqryuD4+Pti4cSP+/vtvbN26FTKZDJ06dUJiYmKl+y9fvhxWVlbCzdXVVanrYIwxXSVfFT42MxaZhZlqO4+8+6tD4w4w0FO684KxCqr9W7Rnz54qH7t48SLWrl0LmUym1MllMhmCgoKwbNkyAGUF1nfu3MH69esxbtw4AGUJ0KVLl7B37164u7vjzJkzmDZtGpydndG7d+9KjxscHIzg4H/7iDt16oTmzZvjhx9+wCeffFJh/9DQUMyZM0e4n52dzUkQY4xVQ0PThnC3ckdcVhzCU8PR3aO7Ws5zIZELoJlqVTsBeuWVVypsi4yMxPz587Fv3z6MHj0aH3/8sVInd3Jygp+fn8K25s2bC/MJFRQUYMGCBdizZw8GDhwIAGjVqhXCw8Px5ZdfVpkA/ZehoSECAwPx8OHDSh83NjaGsbGxUrEzxhgrE+gUiLisOISlhKkvAeIRYEzFalQDlJycjMmTJ8Pf3x+lpaUIDw/HL7/8And3d6WO07lzZ4WRWgDw4MED4TglJSUoKSmBnp5imPr6+kq1NkmlUty+fRtOTk5KxccYY+zF1D0j9NP8p3iQ/gAA0NGlo1rOwXSPUh2pWVlZWLZsGb755hsEBATg+PHj6Nq1a41PPnv2bHTq1AnLli3Da6+9hitXruDHH3/Ejz/+CKBsKHu3bt0wb948mJqawt3dHadPn8aWLVvw1VdfCccZO3YsGjdujOXLlwMAPv74Y3Ts2BFNmzZFZmYmvvjiC8TFxWHSpEk1jpUxxljl1J0AXUy4CABo3qg5GpryhLtMNaqdAH3++edYuXIlHB0dsW3btkq7xJTVrl077NmzB6Ghofj444/h6emJ1atXY/To0cI+f/zxB0JDQzF69Gg8e/YM7u7u+OyzzzB16lRhn/j4eIVWooyMDEyePBmpqamwsbFB27ZtceHChQrdbYwxxmpPPhIs4kkECkoKYGpoqtLjc/cXUwcJEVF1dtTT04OpqSl69+4NfX39KvfbvXu3yoITS3Z2NqysrJCVlQVLS0uxw2GMsXqNiODwpQOe5D/BlUlX0K5xO5Uev9vmbjgTdwY/v/wzJgROUOmxmXZR5vO72i1AY8eO5dmUGWOMVSCRSBDoFIgj0UcQlhqm0gSoRFqCK0lXAHALEFOtaidAmzdvVmMYjDHGNFmg4/8nQCqeEDE8NRyFpYWwMbFBM9tmKj02022izgTNGGNMO6irEFpY/8s1GHoS/shiqsO/TYwxxmpNXgh9K+0WSmWlKjuuMAGiC3d/MdXiBIgxxlitNW3YFOZG5igoLUDk08gXP6Ga5EPguf6HqRonQIwxxmpNT6KH1g6tAaiuGywhKwEJ2QnQl+irfGQZY5wAMcYYUwmhDkhFhdAXE8taf1o7toa5kblKjsmYHCdAjDHGVEJeB6SqFiBhAkSu/2FqwAkQY4wxlSg/Eqyac+w+F88AzdSJEyDGGGMq0cK+BQz1DJFZmIm4rLhaHSu/JF9oSeIEiKkDJ0CMMcZUwkjfCC3sWwCofR3QteRrKJWVwtnCGW5WbqoIjzEFnAAxxhhTmTaObQDUvg6ofPcXL8PE1IETIMYYYyqjqkJoYQZol+Bax8RYZTgBYowxpjKqGApPRFwAzdSOEyDGGGMq09qxNSSQICknCU/yntToGFHPopBekA5jfWMhoWJM1TgBYowxpjLmRubwtvUGUPNuMHnrT5BzEIwNjFUWG2PlcQLEGGNMpWrbDcbrf7G6wAkQY4wxlZInQDdSb9To+cIK8JwAMTXiBIgxxphKCSPBatAClFmYibuP7wLgEWBMvTgBYowxplLyFqCoZ1HIKcpR6rmXEy+DQPCy8YKDuYM6wmMMACdAjDHGVMyugR0aWzQGANxMu6nUc3n4O6srnAAxxhhTuZp2g3H9D6srnAAxxhhTufIrw1eXVCbFpcRLADgBYurHCRBjjDGVq0kCdOfxHeQW58LCyAIt7FqoKzTGAHACxBhjTA3aOJUtinr38V0US4ur9Rx5/U8Hlw7Q19NXW2yMAZwAMcYYUwM3KzfYmNigRFYiDGt/EaH+x4W7v5j6cQLEGGNM5SQSidIrw/MIMFaXOAFijDGmFsosiZGWm4aYjBhIIEEHlw7qDo0xToAYY4yphzKF0BcTy9b/amHfAtYm1uoMizEAnAAxxhhTE3kXWHhqOKQy6XP3Fbq/uP6H1RFOgBhjjKmFj60PTA1MkVeSh4fPHj53X67/YXWNEyDGGGNqoa+nj1YOrQA8vxusqLQI15KvAeAEiNUdToAYY4ypTXUKocNSw1AkLUIjs0Zo2rBpXYXGdJzoCVBSUhLGjBkDW1tbmJqawt/fH9euXRMez83NxfTp0+Hi4gJTU1P4+flh/fr1Lzzujh074OvrCxMTE/j7++Off/5R52UwxhirRHWGwpfv/pJIJHUSF2OiJkAZGRno3LkzDA0NcfDgQdy7dw+rVq2CjY2NsM+cOXNw6NAhbN26FREREZg1axamT5+OvXv3VnncCxcu4PXXX8fEiRMRFhaGIUOGYMiQIbhz505dXBZjjLH/V34kGBFVug8XQDMxSKiq38g6MH/+fJw/fx5nz56tcp+WLVti5MiRWLhwobCtbdu26N+/Pz799NNKnzNy5Ejk5eVh//79wraOHTsiICCg0tajoqIiFBUVCfezs7Ph6uqKrKwsWFpa1uTSGGOMASgsLYT5MnNISYqE2QlwsXRReJyI4PyVM1JzU3F6/Gm85P6SSJEybZCdnQ0rK6tqfX6L2gK0d+9eBAUFYcSIEbC3t0dgYCA2bNigsE+nTp2wd+9eJCUlgYhw8uRJPHjwAH379q3yuBcvXkTv3r0VtvXr1w8XL16sdP/ly5fDyspKuLm6utb+4hhjjMHEwAR+dn4AKq8DisuKQ2puKgz0DBDkHFTX4TEdJmoCFBMTg3Xr1sHb2xuHDx/GO++8gxkzZuCXX34R9vnmm2/g5+cHFxcXGBkZISQkBN999x1eeqnqbwmpqalwcHBQ2Obg4IDU1NRK9w8NDUVWVpZwS0hIUM0FMsYYe24dkLz7K9AxEGaGZnUaF9NtBmKeXCaTISgoCMuWLQMABAYG4s6dO1i/fj3GjRsHoCwBunTpEvbu3Qt3d3ecOXMG06ZNg7Ozc4VWnpoyNjaGsbGxSo7FGGNMUaBjILbc3PLcBIiHv7O6JmoC5OTkBD8/P4VtzZs3x65duwAABQUFWLBgAfbs2YOBAwcCAFq1aoXw8HB8+eWXVSZAjo6OSEtLU9iWlpYGR0dHNVwFY4yx53neUHj5EhicALG6JmoXWOfOnREZGamw7cGDB3B3dwcAlJSUoKSkBHp6imHq6+tDJpNVedzg4GAcP35cYdvRo0cRHBysosgZY4xVV4BjAICyep9nBc+E7bnFubiZehMAJ0Cs7omaAM2ePRuXLl3CsmXL8PDhQ/z+++/48ccfMW3aNACApaUlunXrhnnz5uHUqVOIjY3F5s2bsWXLFgwdOlQ4ztixYxEaGircnzlzJg4dOoRVq1bh/v37WLJkCa5du4bp06fX+TUyxpiuszKxQhObJgDK1gWTu5p0FVKSwtXStcLoMMbUTdQEqF27dtizZw+2bduGli1b4pNPPsHq1asxevRoYZ8//vgD7dq1w+jRo+Hn54cVK1bgs88+w9SpU4V94uPjkZKSItzv1KmTkEy1bt0aO3fuxF9//YWWLVvW6fUxxhgrI+8Gu5FyQ9jG9T9MTKLWAAHAoEGDMGjQoCofd3R0xKZNm557jFOnTlXYNmLECIwYMaK24THGGFOBQMdA7IrYpVAIfSGREyAmHtGXwmCMMab9hKHw/18ILSMZLiZwATQTj+gtQIwxxrSfvAssMj0S+SX5iMuMQ0ZhBkwNTNHaobXI0TFdxC1AjDHG1M7JwgkODRwgIxlupd0S6n/aNW4HQ31DkaNjuogTIMYYY3WifDcYL4DKxMYJEGOMsTpRfmV4LoBmYuMaIMYYY3WijVMbAMCJ2BOIzogGAAS78gS1TBzcAsQYY6xOyFuA5MmPi6ULbExsxAyJ6TBOgBhjjNWJsNQwSCAR7idmJ8JjjQd2R+wWMSqmqzgBYowxpna7I3bjtR2vgUAK25Oyk/Dq9lc5CWJ1jhMgxhhjaiWVSTHz0MwKyQ8AYdusQ7MglUnrOjSmwzgBYowxplZn488iMTuxyscJhITsBJyNP1uHUTFdxwkQY4wxtUrJSXnxTkrsx5gqcALEGGNMrZwsnFS6H2OqwAkQY4wxterq1hUuli4KI8DKk0ACV0tXdHXrWseRMV3GCRBjjDG10tfTx5qQNQBQIQmS318dshr6evp1HhvTXZwAMcYYU7thzYdh52s70diyscJ2F0sX7HxtJ4Y1HyZSZExXSYio4rhEHZednQ0rKytkZWXB0tJS7HAYY0xrSGVSnI0/i5ScFDhZOKGrW1du+WEqo8znN68FxhhjrM7o6+mju0d3scNgjLvAGGOMMaZ7OAFijDHGmM7hBIgxxhhjOocTIMYYY4zpHE6AGGOMMaZzOAFijDHGmM7hBIgxxhhjOocTIMYYY4zpHE6AGGOMMaZzeCboSshXB8nOzhY5EsYYY4xVl/xzuzqrfHECVImcnBwAgKurq8iRMMYYY0xZOTk5sLKyeu4+vBhqJWQyGZKTk2FhYQGJRKLSY2dnZ8PV1RUJCQk6udCqrl8/wK8BX79uXz/Ar4GuXz+gvteAiJCTkwNnZ2fo6T2/yodbgCqhp6cHFxcXtZ7D0tJSZ3/xAb5+gF8Dvn7dvn6AXwNdv35APa/Bi1p+5LgImjHGGGM6hxMgxhhjjOkcToDqmLGxMRYvXgxjY2OxQxGFrl8/wK8BX79uXz/Ar4GuXz9QP14DLoJmjDHGmM7hFiDGGGOM6RxOgBhjjDGmczgBYowxxpjO4QSIMcYYYzqHEyDGGGOM6RxOgBhjjDGmczgBUgNdmllAJpNVul2XXgPGGJMjIuH9T1ffB8PDw1FaWgqgfr8GnACpUHh4OHJzc1W+gGp9JZPJhMXmIiIi8PDhQ0RHRwOAzrwGjFWmPr/pM9WSfwmUf+BLJBLExMQI/9c1+fn56NGjB95//30A9fs14ARIRRYuXIjZs2dj//79kEqlWv8GSERC8rNgwQIMHz4cnTp1Qp8+fbBgwQKRo6tb8p91eno6Hj9+XOlj2kx+jSkpKYiIiEBWVhZKSkoAVN1CqE3k15+QkIB79+4hISEBhYWFIkdVt+SvQXR0NB4+fIi4uLgKj2krPT09PHjwAPPmzQMA7NixA/7+/oiMjBQ5srpRvsULAMzMzLBmzRqcOXMGR44cETGyaiBWa6GhodSoUSM6cuQIpaenKzwmlUpFiqpufP7559SwYUM6evQoHTp0iH744Qdq0KABTZ48WezQ6tTu3bupXbt25OrqSu+99x5dvXpVeEwmk4kYmXrJr23Pnj3UsmVLcnR0pPbt29PixYuFvwVt/huQX//u3bvJ29ubmjdvTk2aNKEJEybQlStXRI6ubu3YsYNcXV3JycmJOnfuTD/88IPwmDb/DRARHT9+nCQSCfXr14/09fVp8+bNRKT9113e5cuXKTMzk4iIkpOTaciQITRz5kwqKCgQObKqcQJUS1evXqVmzZrRxYsXiYgoKyuLHjx4QD/88ANFRUURkfZ8ACQkJCjcLy4upqFDh9Inn3yisP3IkSNkaGhIa9asqcvwRHPt2jVycHCgxYsX06pVq6hJkyY0ePBgOnLkiLCPNr8R/vPPP2RhYUGrVq2itLQ0mj17Njk7O9OUKVPoyZMnRKQ9fwOVOXXqFFlYWNDatWuJiOjrr78mQ0ND2rhxo8iRqZ/89zopKYm8vLzop59+ol27dtGsWbPIzc2Nvvzyywr7ahv5dS1cuJAkEgn16NGD8vLyRI6qbj148IAkEgm1aNGCfvzxR0pJSaEzZ86QqakpHT58mIjq53sAJ0C1dPv2bXJ3d6cTJ07Q7du36b333iNvb2/y9PQkExMTioyMFDtElZgwYQINHTpUYVteXh41a9aMZs2aJWwrLS0lIqJ3332XhgwZQkVFRVr7xkdEFBUVRV9++SUtXbpU2Hbr1i1q3749DRo0iI4ePSps14bX4cyZMwr309LSqHfv3rRixQoiIkpPTyc3Nzfq0KEDtWzZkt5++22tbQmSX8/7779PEyZMICKixMREatKkCU2dOlXYT/6tWFtduHCB5s+fT9OnTxdek4SEBFq0aBE1btxY65OgkpISIiJasmQJzZ07l/T19Wny5MmUnJxc6f7a+Bqkp6dT+/btycrKij766CPq2bMnnTt3jhYtWkTNmjWjlJQUsUOsFCdANRQREUF5eXmUmJhIgwYNoubNm5OZmRlNnTqV/vjjD3r69Ck1b95ca1pBMjIyqLi4mIiInj17JmxfsmQJtWnTRmgBkwsNDaWePXtq5R87UdmbWHp6Orm6upKJiYnCBx4R0c2bN6ldu3b0yiuv0P79+0WKUrWOHz9OFhYW9OTJE+HnWlpaSr/++ivdu3ePHj9+TD4+PsJrMW7cOLK2tqZRo0YJLUGaqnzyJv87kCf777zzDn377beUmZkptHzJX5+//vqLdu7cKeyrbXJycmjq1KlkZWVFvXr1UnhMngR5eHhUaCXWBvKfsbyLR35///79QhKUmpoq7F++W1xbJCYmUlpaGhERhYeHk5+fH33++ee0efNmatCgAQ0bNoycnJzo448/pqKiIpGjrYgToBrYtWsXmZub0/bt24mIKDIykv766y86efKk8OaYl5dH7dq1o99//13MUFXup59+IicnJ3rw4AEREZ0+fZq6d+9Oo0ePpvPnzxNR2TfePn360KRJk8QMVW3KJ3UnT56kpk2bUnBwcIU3uFu3bpG3tzeNHDlSK5rECwoKhDe7R48eCdvl1/b555/TwIEDhRafNWvWUPPmzWn48OGUlJRU9wGrWPku4AMHDtD69euJqCzZd3V1JRcXF5oxY4bQIlBcXExjxoyhDz/8UHhf0Bbl/wauX79OU6dOJWNjY9qyZYvCfomJiTR37lzy8/Oj9PR0rflCJL+Ow4cP07hx46h///40adIkiomJISKiQ4cOkYGBAU2ZMoWuX79On3zyCVlZWSl8edB0ERER1Lp1axo3bhxdu3aNiIhWrFhBs2bNIplMRleuXKFx48aRnp4eBQYGUm5ursgRV8QJUA298sor5OHhQTt37qTCwkJhe0FBAcXExNDAgQMpKChI6775paWlUUBAAPn7+9PDhw+JqKwAtk+fPuTg4EBBQUHC4/I3fW35gy/f6kH0b6vA8ePHycPDg8aMGUM3btxQeM6dO3eEN0VtERMTQxKJhL744guF7XPnzqX27dsLb3Rz586llStXVhgYoIlycnIoKCiIevfuTTt37iSJREI7duwgIqLs7Gzq27cvWVtbU05ODhGVJT+hoaHUuHFjrekGJ1Js9ZAnekRlNSCTJ08mX19f2rp1q8JzkpKSNL4FsDJ79uwhU1NT+uijj+jLL7+krl27kqWlpdDqc+zYMWrYsCG1bt2aHBwchCRBm6xdu5bGjBlDFhYW9PPPP9Nvv/1Gffr0Ebr+09LS6OjRo8JnRX3DCVAtDB8+nNzc3GjHjh2Un59PRETff/899e/fnzp16lShqVzTVFWz8fTpU2rTpg35+vpSdHQ0ERHdv3+f/v77b1qwYAGtW7dOeHMs/yapyeRv/MeOHaNp06bR6NGj6bPPPhNaRA4fPkweHh40evRoCg8PFzNUtSspKaElS5aQkZGRQhfvd999R+3bt6fhw4fTm2++SQ0aNBBaCjVdYWEhHT9+nOzt7cnY2Jh+/fVXIipLdGQyGR0/fpxatGhBTk5O1KtXL+rXrx/Z29tXSIg1mfxv4MCBAxQSEkKdO3emAQMGCKPdoqKiaMqUKeTj46N1Ld//lZ6eTp06daKvv/6aiIji4+PJ1dVVaPWWv1YRERF0+vTpCgNINJH8mvLz8xW+1OTn59Pq1avJ0dFRqIH19/fXiC8+nABV03fffUcnT56skBQMHTqUHB0daefOnURU1u3x66+/CkmPpiYA5a/z0qVLdOjQIbp16xZlZGQQUdkbgDwJqiq719TEryp79uwhExMTmjRpEvXp04eCgoLI3d2d4uLiiKhs9Ju3tze9/PLLdOvWLZGjVR35G9+9e/fo7NmzFB8fT0REX375JUkkElq9ejURlSUJixcvpmHDhlH//v01+jWoLPl/9OgR2djYkLW1Nb3yyisKj8lkMnr27Bl99tlnNG/ePFq9erXw5UCb7N+/nwwNDen999+nTz/9lLp160bOzs5CQnj37l169913yd7eXigR0Ebx8fHk5uZGiYmJlJKSQo0bN6YpU6YIj//555/Ce6U2kL8H7Nu3j3r37k2+vr40cOBA+vXXX4Uv/6dPn6b33nuPgoODSSKR0NKlS+t96z8nQNUUEBBALi4udP78+QpvjgEBAdSqVSvasmWLwg9cExMAmUymcA0ffPABOTs7k4eHBxkZGdGoUaOEYY3p6enUtm1b8vPz05pv+lV58uQJBQQE0Oeffy5su337NvXt25c8PT3p8ePHRFTW99+6dWutqHkpb8+ePWRubk5eXl5kbGxMGzZsoLS0NPrqq69IIpHQqlWriOjfxKE+z/1RXUlJSUJd19atW2nu3LkUHR1NR44cIS8vLxowYICwr6Z+0akumUxGubm51KtXL5o/f77CY2+99RY5OzvTzZs3iaisJmj27Nn1ttujNi5fvkzPnj2j/Px8GjBgAG3YsIHc3Nzo7bffFn4H4uPjafTo0fTPP/+IHK1q/fPPP2RkZETz58+n9evXU9++fal9+/b00UcfCUnQ48eP6cKFC9SjRw+KiIgQOeIX4wToBX777TehObdnz57k4eFB586dE5IbmUxGI0eOJGtra3r11VfFDLXW/ttM+8MPP5CdnR2dPn2aMjIyaP/+/dS3b18aOHCgMBz6yZMnFBQURC1bthSGfWrbcGciotjYWHJwcFCY26e0tJTCw8OpTZs29M033wjXrQ0Fz3JSqZTS09OFie2ioqLo008/JYlEQitWrKCUlBT66quvyNjYmJYtWyZ2uCohk8mooKBAGMW3fPlykkgkwsR+hYWF9Ndff1HTpk1p0KBBwvO+//572rBhA0ml0nr/zfd5ysdePrGTSqUUGBgodHuWr30MDg6mESNGCPfr44gfZf23aPfhw4fUuHFjun//PkmlUho5ciRJJBKF6yYq+9Ko6V+Cnj59SkRlvwtSqZTy8vJo0KBBCslvcXExLViwgNq1a6exrX2cAD3HnTt3KDAwkFq1akUHDhwgIqLu3buTh4cHnTlzRvige+utt+jevXsa/cH/7rvv0rx584jo35arCRMm0NixYxX2O3XqFLVt21bYl6isJSgwMJA6d+5cdwHXscLCQmrTpg199NFHCttlMhm1b9+epk+frrBN05Uvds3Pz6cFCxYoTH+wevVqhSTos88+o4YNGyrso+mioqLI1dWVJBIJLV68WOGx8klQq1ataOrUqSSRSOju3bviBKsi8p+7/AOQqGzuJ/kIzx49elBISIjwmDwJmjVrlkIyqOm+//57atmypUISEx0dTZ6enkLdX05ODgUGBlJAQACtXLmStmzZQm+//TZZWlpqdB3g1q1bqVevXhV+l7t3705z5swhon8/I0pLS+mll16ikSNH1nmcqsBrgVVh3rx5WLx4MUxNTZGYmIj33nsPf//9N06ePAlvb2+MGzcOQ4cORadOnXDp0iU0a9YMenp6Grv2Ud++ffHZZ58BADIzM4XtOTk5AP5d06lbt254/fXX8fPPPyMzMxNEhIYNG2LFihVIT08XFkPVZPT/69qUlpaioKAAAGBsbIzu3bvj6NGj2L17t7CvRCJB48aNYW1tLayJU58X/6suiUSCv//+G0OGDEFQUBB2796NhIQE4fGZM2fi66+/xqJFi7B+/XpMnDgRUVFRsLGxETFq1SkpKUHDhg2hr68Pe3t7REZG4vLly8LjxsbGGDBgAH799Vf4+Pjg8ePHuHnzJvz8/ESMuvYkEgmePn2KQYMGYdmyZdi7dy+6deuG7OxsAEBoaCiioqIwY8YMAGWvAwA8ffoUFhYWKC0t1Yq1v/r06YOcnBy88cYbSE5OBgBkZWXB0NAQtra2kEqlMDc3x8mTJ+Hn54ft27dj5cqVSEtLw7lz59C6dWuRr0B58p9bbm4uiouLsWTJEkRERAAAioqKYGVlhXv37gEA9PX1IZVKoa+vj379+iEhIQHFxcWixV5jYmZf9dWmTZvI2tqarl+/Ts+ePaOUlBTq27cvBQUF0d69e4mobL6Dt99+m959912hmVgTW4D+21rxyy+/UL9+/SgpKYl27NhBEomEzp49q7DPn3/+SR07dqTs7Gxh25dffkne3t4K3xw1UfmRLqNGjaLmzZvT+++/T8ePH6fCwkIaNGgQdezYkWbOnEnbt2+n6dOnk6WlpUb0dyvj6tWrZGlpSVOnTqXx48eToaEhzZw5U2H+HyKi5cuXk42Njcb/3KuSk5NDN2/epKZNm9KwYcMqTPgpJ6+B0GTya4iLi6NPP/2UXFxcyMTEhLZt2ybsk52dTd988w15enpSjx49aOHChTRu3Dhq0KAB3b59W6zQ1SI2Npa8vLyoS5culJqaShcuXCBfX99K673y8vIoJydHoVtQ05T/3d6yZQv16NGDhg4dKvxcb926RaampjRjxgyFz7qxY8fS8OHDNXKuK06AKvHRRx9Rly5dSCqVCj/oxMRE6tChgzD3j5z8cW0pgvz+++8pODiY3njjDUpKSqKZM2eSlZUVHThwgB49ekQZGRnUp08fGjRokELytHv3bo1PAuTX8/fff5OZmRktXLiQfv31V+revTt5enoK1//BBx9Qx44dydvbm7p27UphYWHiBq5iDx8+pEWLFtHy5cuFbd9//z25uLjQ/PnzKyRB2tLtJf/5h4eH07Zt22jLli3CtV6+fJmaNm1KI0aMoAsXLhBR2QSIS5YsUXiuptqyZQs5ODgIc9gcOnSIJBIJOTg4KPweEJUlhadPn6ZBgwZRr1696NVXX9W65EdOngT17duX9u7dSwEBAbR27VrauXMnnTx5kvbs2UMbN26kO3fuiB1qrZw4cYLs7OwUfta//PKLkATJR3Tu2rWLTE1NqWfPnjR+/HgaN24cmZuba+yIT06AypG/iX388ccUFBQkjGSRZ7YnTpwgMzMz6tGjB/3xxx+ixalumzdvpq5du9Jrr71GV65cofnz55OpqSm5uLiQn58fBQQEaPwcR3IHDhwQRq/IZDJ6/PgxdevWTZjfIz8/n+zs7Gj27NkVWvjS0tLq5eymtZGVlUVBQUHUqFEjWrBggcJj3377LTVu3Jg++ugjhckdNf3Dv7ydO3eSq6srtWvXjrp160aGhoa0b98+Iipb9LZFixbUsWNH6tu3L5mZmVXZIqQp5D+7kydPUnBwMLVs2ZIeP35M6enptHfvXlq+fDn5+vpWqIEqTxO/+VdF/nrcv39f+EIXGxtLTZs2FRb7bNWqFbVs2ZLatm1L3t7e5OPjQ/fv3xcz7Fq7f/8+vf/++9SiRQtauXKlsP2XX36h7t2709ChQ+nevXtEVDYdxtixY2no0KE0duxYjU5+OQGqxK1bt0hfX1/4did36NAhGj58OPXs2ZN69+6tFSMdyiv/QbZx40bq1q0bjRw5kp49e0Y3b96kHTt20I4dOzR+jiO51NRU8vT0FIrYicpGfrRt25bu379PMTEx1LhxY5o8ebLwnEOHDlFUVJRYIdeJGzdukLe3N3Xu3LnCm9u6devIxMSEli5dqvE///+6evUqNWzYkH788UciKlviRiKR0Mcffywkv+Hh4bRgwQKaPn26xhc8E5XN3UJU9rd/8eJF6tKli5AEEZXNfbR48WLy9fVVWPB369atQmG0tiTA8uvYtWsX+fn50aJFi4RFPGNjY6lt27bk7+9P0dHRVFJSQiUlJcIgAU315ZdfCl/i4uLi6MMPPyQfH58qkyD5l0V50qvpyS8nQFXYtGkTGRoa0rx58+jatWsUHR1NAwcOpM8++4zu3btHEolEYaVvbfHfJKhLly40cuRI4Rv/f5eD0HTXr1+ndu3a0aRJk+j27dtUUFBAfn5+tH79emratClNmjRJuNbY2FgaNWqU1s3vUZmbN29SQEAATZkypULz/k8//aTx8z79d1V7IqIdO3YIQ5pjYmLIxcWF3nnnHeFxeVdfaWmpRtb7/Zd8cVv5qCaZTEbnzp2jLl26kK+vr9AdFhcXR0uWLCFfX1+aMGECffTRRySRSLRynp/Dhw+TqakprVu3TkgC5WJjY6lJkybUo0cPYTJQTRYVFUVdu3ZVKF2Iiop6bhI0YsQIhRFump78cgL0HDt37iR7e3tycXGhxo0bU2BgIBUUFNCjR4/I29tbyIa1zX+ToK5duyokQdrmxo0b1KZNG5o4cSIlJyfTt99+SxKJRGGiOyKiBQsWUMuWLbXiza865K/LpEmTtKK1Q66yVe2JiL766ivq3LkzRUdHk5ubG02ZMkVIdPbu3UvTpk3Tqi7P8ovblv+CI0+CmjdvLiRBCQkJ9M0331D79u0pODhYq5b4ICq77sLCQnrzzTdp7ty5wjYixS978tnAQ0JCNP5LYGlpqbB23dmzZ4X/V5UE/frrrxQYGEhvvvmm1vR+cAL0AomJiXTx4kU6c+aM8GY4f/588vX1FZpHtVH5D4bNmzfTSy+9RB9++CEVFhZqfNZfmRs3blBAQABNmjSJjh07Ru+//z7p6+vT559/Tp9//jm98847ZGFhoXUFzy9y48YNat++PY0aNUrji9zlqlrV/sqVK9StWzeysbGh8ePHE9G/gxxmz55Nr732GmVlZdV9wGomX9xWPst5VUlQYWEhSaVSyszMFDNcterUqRPNnj270sfkX3zi4uI0vhu8/Ht4cnIy9ejRg5o2bSok+FUlQdu2baswCEKTcQKkhDt37tCbb75Jtra2OvFBWP6P5P3336cuXbpoTeZfmRs3blDbtm3p7bffpiNHjtA333xDLVq0oA4dOtDrr7+u0cV+tSFPDOQzfWuL/65qX1RURJMmTSJHR0f69ttvKTc3lxITE2n+/Plka2ur8SN9qlJ+cVt58X/5JMjf319rv+yVb+XJzc2l3r170xtvvCFsk++TkJBA8+fP17puvxs3btCUKVNox44d1KlTJ2rTpk2FJKhFixYV6mG1BSdA1VRSUkI3btyguXPnau0bYWXkbxBLliyhJk2aaPW3P6KymqCgoCCaNGkSJScnK8yIrMu08frLf/B/9dVXRFR2ncOHDyd/f38yNzenTp06kZeXl1Z1+bxocdvySdD58+epZcuW1KFDB62oe5KTvwby2fzl8/fs27dPISmWCw0NpaCgIKHlUFt89dVX1LZtW7p27RqdO3eOWrduTW3btlVIgqZPn07t2rWjp0+fal3rPydAStL0qveakMlktH37do2e3l0ZN27coHbt2tHIkSOFZFfb/vB10Ys++L/88ksiKmsJCg8Pp82bN9O5c+coMTFRtJjV5UWL28qTIKlUSpcuXaLY2FhR41Ul+e/BP//8Q0OGDKEePXrQiBEjhFb9NWvWkEQioWHDhtHYsWNp9OjRZGlpqRVJsPzay49c69KlC/Xu3ZuIyiZDDAgIUEiCoqOjtS7xk+MEiLFKaGu3j66r7qr22koXF7etzN9//y2sbD5jxgwaMGAAmZiYCHM+HT9+nMaMGUODBw+mqVOnatUggEOHDtGYMWPo8OHDRFRW09SkSRNasWIFEZVNjdC+fXtq0qSJVhX9V4YTIMaqoI3dPrpKmQ9++QeBNqnp4rbp6ela1/pZXFxM/fv3V1jZPD8/n2bNmkXGxsbCrMbyekdtmu9KJpPR5MmTSSKRUMOGDWnx4sUUExNDn332Gb366qt069YtkslkdOjQIerevbvWjvyVMxB7LTLG6isTExOxQ2C1RP+/OG1xcTFMTU3RrVs3jBgxAjY2Nvjoo49gbm6O2bNnAwDGjRuHgoICfP7555gyZYrWLOwK/Lu47bp165CQkACZTIaRI0cK1zhz5kwAwAcffICCggK88847mDp1Kho2bChm2Cr3999/IyoqCo8ePcLLL78MoOx3xNjYGJ9++inu3LmDtWvX4vvvv4eeXtla4fr6+mKGXGtUboFmiUSCSZMmITc3Fy1btsSePXuQlpaG0tJS3L17F0ePHoW/vz969uyJrl27wszMTOTo1YtXg2eMaS1dX9Ve7tq1axg7diw8PT3Rvn17REdHY+PGjYiLixP2mTlzJpYuXYq1a9fCwMBA65Kf69evY8KECXB3d0ezZs1w4MAB5OfnQyKRQCKRoEGDBnB0dMTTp09haGgIA4Oy9gF58qCpJBIJTpw4gZ9++gkAEBQUBFtbW0RHR+PEiRNo1aoVAOD+/ft4//33cfHiRRgaGmp98gNwAsQY02L8wQ9ER0dj3759CA0Nxbp167Bp0yasWbMGu3btwvr16xVei/nz5yM6Ohq2trYiRqx6Dx8+xN69ezF58mSMGDEC/fv3x5MnT7Bq1SoUFhYKSY6BgQFsbGxQUlICIhI5atWQSqW4fPkypkyZgrFjx+LixYtYu3YtwsPDsXr1arzzzjtYtWoVpk2bBmdnZ9jZ2Ykdcp2RkLb8lBljrJzo6Ghs2bIFpqammD9/PgBg3bp1WLZsGcaMGYOpU6fC3d1d2D8jI0PrWn6ys7PRq1cvPHr0CFOmTMFnn30mPPbdd99h+fLlGD9+PCZOnAhPT08Ail0m2kD+GsTFxWH06NH4+uuvUVpaitDQUJw+fRoWFhbo3bs3IiMjsWvXLly6dAktWrQQO2yVu3XrFubNm4fc3Fy0a9cOISEhWL9+PT744AN06tQJAJCZmQlra2txA61LItYfMcaYWuj6qvbl6erituXJX4OAgAC6fv06EZVNdPjLL7/QG2+8Qe3bt6fhw4cLBdDaKjU1lbZs2UIBAQHUoEED8vT0pI8++kjssETDCRBjTCvxB/+/tH1x2+q4efMmtWrViiZNmlRhHcf8/HydmuOtuLiYZs+eTYaGhmRvb0/Z2dlihyQK7gJjjGmtW7duYdy4cWjfvj1mzJih0LXx888/46WXXoK3t7eIEdadsLAwTJo0CW3atMHs2bPh5+cndkh1rvxrMGvWLK3s6noRKtfFeezYMXh7eyt0BesSToAYY1qNP/j/FRYWhqlTp6JJkyZYvHgxfH19xQ6pzvFroH11XjXFo8AYY1otMDAQP/30E27duoVPPvkE9+/fFzsk0QQGBuLbb79FSkoKrKysxA5HFPwaaP7QflXhFiDGmE64evUq5s2bh23btsHJyUnscERVWFio8xN98mvAOAFijOkM/tBjjMlxAsQYY4wxncM1QIwxxhjTOZwAMcYYY0zncALEGGOMMZ3DCRBjjDHGdA4nQIwxxhjTOZwAMcYYY0zncALEGGOMMZ3DCRBjjDHGdA4nQIwxrSWVSiGTycQOgzFWD3ECxBirUx4eHli9erXCtoCAACxZsgREhCVLlsDNzQ3GxsZwdnbGjBkzhP2Kiorw/vvvo3HjxmjQoAE6dOiAU6dOCY9v3rwZ1tbW2Lt3L/z8/GBsbIz4+Hh4eHhg2bJlmDBhAiwsLODm5oYff/xRIYYPP/wQzZo1g5mZGZo0aYKFCxeipKREeHzJkiUICAjAxo0b4ebmBnNzc7z77ruQSqX4/PPP4ejoCHt7e3z22WcKx83MzMSkSZNgZ2cHS0tL9OzZEzdv3lTdC8oYqxEDsQNgjDG5Xbt24euvv8Yff/yBFi1aIDU1VSFZmD59Ou7du4c//vgDzs7O2LNnD0JCQnD79m14e3sDAPLz87Fy5Ur89NNPsLW1hb29PQBg1apV+OSTT7BgwQLs3LkT77zzDrp16wYfHx8AgIWFBTZv3gxnZ2fcvn0bkydPhoWFBT744APh/NHR0Th48CAOHTqE6OhovPrqq4iJiUGzZs1w+vRpXLhwARMmTEDv3r3RoUMHAMCIESNgamqKgwcPwsrKCj/88AN69eqFBw8eoGHDhnX10jLG/osYY6wOubu709dff62wrXXr1rR48WJatWoVNWvWjIqLiys8Ly4ujvT19SkpKUlhe69evSg0NJSIiDZt2kQAKDw8vMI5x4wZI9yXyWRkb29P69atqzLOL774gtq2bSvcX7x4MZmZmVF2drawrV+/fuTh4UFSqVTY5uPjQ8uXLyciorNnz5KlpSUVFhYqHNvLy4t++OGHKs/NGFM/bgFijNUbI0aMwOrVq9GkSROEhIRgwIABGDx4MAwMDHD79m1IpVI0a9ZM4TlFRUWwtbUV7hsZGaFVq1YVjl1+m0QigaOjIx4/fixs+/PPP7F27VpER0cjNzcXpaWlsLS0VDiGh4cHLCwshPsODg7Q19eHnp6ewjb5cW/evInc3FyF+ACgoKAA0dHRyrw0jDEV4wSIMVan9PT0QEQK2+S1Nq6uroiMjMSxY8dw9OhRvPvuu/jiiy9w+vRp5ObmQl9fH9evX4e+vr7C883NzYX/m5qaQiKRVDivoaGhwn2JRCIUSF+8eBGjR4/G0qVL0a9fP1hZWeGPP/7AqlWrXniM5x03NzcXTk5OCnVKctbW1hW2McbqDidAjLE6ZWdnh5SUFOF+dnY2YmNjhfumpqYYPHgwBg8ejGnTpsHX1xe3b99GYGAgpFIpHj9+jK5du6o0pgsXLsDd3R0fffSRsC0uLq7Wx23Tpg1SU1NhYGAADw+PWh+PMaY6nAAxxupUz549sXnzZgwePBjW1tZYtGiR0KKzefNmSKVSdOjQAWZmZti6dStMTU3h7u4OW1tbjB49GmPHjsWqVasQGBiIJ0+e4Pjx42jVqhUGDhxY45i8vb0RHx+PP/74A+3atcOBAwewZ8+eWl9r7969ERwcjCFDhuDzzz9Hs2bNkJycjAMHDmDo0KEICgqq9TkYYzXDw+AZY3UqNDQU3bp1w6BBgzBw4EAMGTIEXl5eAMq6hTZs2IDOnTujVatWOHbsGPbt2yfU0GzatAljx47F3Llz4ePjgyFDhuDq1atwc3OrVUwvv/wyZs+ejenTpyMgIAAXLlzAwoULa32tEokE//zzD1566SW89dZbaNasGUaNGoW4uDg4ODjU+viMsZqT0H874xljjDHGtBy3ADHGGGNM53ACxBhjjDGdwwkQY4wxxnQOJ0CMMcYY0zmcADHGGGNM53ACxBhjjDGdwwkQY4wxxnQOJ0CMMcYY0zmcADHGGGNM53ACxBhjjDGdwwkQY4wxxnTO/wEw22oddfMz/wAAAABJRU5ErkJggg==\n", - "text/plain": [ - "
" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "df['upvotes_normalized'] = df['total_upvotes']/df['num_posts']\n", - "\n", - "df.plot(x='username', y='upvotes_normalized', marker='o', color='green')\n", - "plt.xticks(rotation=45, ha='right')\n", - "plt.ylabel(\"Number of upvotes (normalized)\")\n", - "plt.gca().get_legend().remove()\n", - "plt.show()" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "You've been working with data taken at a single snapshot in time from when you ran the last query. Run the same query again, and store the output in `response2`, which you will compare with the previous results:" - ] - }, - { - "cell_type": "code", - "execution_count": 16, - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "
\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "
num_poststotal_upvotesusername
40428166willow
41829413jojo
41929202mia
41929456miette
42829472gus
43330160milton
44031212leon
44331063rocket
" - ], - "text/plain": [ - "" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "response2 = sql_client.sql_query(sql)\n", - "response2.show()" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "Normalizing the data also helps you evaluate trends over time more consistently on the same plot axes. Plot the normalized data again, this time alongside the results from the previous snapshot:" - ] - }, - { - "cell_type": "code", - "execution_count": 17, - "metadata": {}, - "outputs": [ - { - "data": { - "image/png": "iVBORw0KGgoAAAANSUhEUgAAAkAAAAHMCAYAAAA9ABcIAAAAOXRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjcuMCwgaHR0cHM6Ly9tYXRwbG90bGliLm9yZy88F64QAAAACXBIWXMAAA9hAAAPYQGoP6dpAAC6DklEQVR4nOzdd3iTZffA8W+SbroodNKWsilQoOxV9ihLkCUqAoKoiALixNetPxDfVwW3Iut9FZSpAlKWjLJX2WWVQgdtoYXuneT3R22ktkDTJk3Sns915ZI+ffI8J0ibk/s+97kVWq1WixBCCCFEDaI0dQBCCCGEEFVNEiAhhBBC1DiSAAkhhBCixpEESAghhBA1jiRAQgghhKhxJAESQgghRI0jCZAQQgghahwrUwdgjjQaDTdu3MDJyQmFQmHqcIQQQghRDlqtloyMDHx8fFAq7z/GIwlQGW7cuIGfn5+pwxBCCCFEBcTGxuLr63vfcyQBKoOTkxNQ9Bfo7Oxs4miEEEIIUR7p6en4+fnp3sfvRxKgMhRPezk7O0sCJIQQQliY8pSvSBG0EEIIIWocSYCEEEIIUeNIAiSEEEKIGkdqgIQQQogKUKvVFBQUmDqMGsXa2hqVSmWQa0kCJIQQQuhBq9WSmJhIamqqqUOpkVxdXfHy8qp0nz5JgIQQQgg9FCc/Hh4eODg4SMPcKqLVasnOzubmzZsAeHt7V+p6kgAJIYQQ5aRWq3XJT506dUwdTo1jb28PwM2bN/Hw8KjUdJgUQQshhBDlVFzz4+DgYOJIaq7iv/vK1l9JAiSEEELoSaa9TMdQf/cyBSaEEFVErVETHhNOQkYC3k7ehPiHoFIaZkWLEEI/kgAJIUQVWB+5nllhs4hLj9Md83X2ZVHoIkYFjjJhZELUTDIFJoQQRrY+cj1jVo8pkfwAxKfHM2b1GNZHrjdRZMJU1Bo1u6/tZtWZVey+thu1Rm3SeCZPnszIkSNNGkNVkxEgIYQwIrVGzaywWWjRlvqeFi0KFMwOm82IZiNkOqyGqOrRwAfVzLzzzjssWrQIrbb0v1Fjy83N5dlnn+X48eNERkYybNgwfv311yq5t4wACSGEEYXHhJca+bmbFi2x6bGEx4RXYVTCVEwxGpiQkKB7LFy4EGdn5xLHXn75ZVxcXHB1dTX4vR9ErVZjb2/PzJkz6d+/f5XeWxIgIYQwooSMBIOeJ8yPVqslKz/rgY/03HRmbpl5z9FAgFlbZpGem16u65V3xMbLy0v3cHFxQaFQlDjm6OhYagqsd+/evPDCC8yePZvatWvj6enJ4sWLycrK4sknn8TJyYnGjRuzZcuWEvc6e/YsgwcPxtHREU9PT5544gmSk5PvGVutWrX45ptvmDZtGl5eXuV6PYYiU2BCCGFE3k7l61Zb3vOE+ckuyMZxvmOlr6NFS1xGHC4LXMp1fubcTGrZ1Kr0fe9lxYoVvPrqqxw5coRffvmF6dOns2HDBh5++GHeeOMNPvvsM5544gliYmJwcHAgNTWVvn378tRTT/HZZ5+Rk5PDa6+9xrhx4/jzzz+NFmdFyQiQEEIYUYh/CL7Ovigouw5DgQI/Zz9C/EOqODIh7q9Nmza8+eabNGnShLlz52JnZ0fdunWZNm0aTZo04e233yYlJYXTp08D8OWXXxIcHMy8efNo3rw5wcHBLF26lF27dnHp0iUTv5rSZARICCGMSKVUsSh0EWNWjyn1veKkaGHoQimAtmAO1g5kzs184Hl7r+9lyMohDzzvj8f+oGf9nuW6rzG1bt1a92eVSkWdOnUICgrSHfP09ATQ7c116tQpdu3ahaNj6dGwqKgomjZtatR49SUJkBBCGNmowFF8P/x7pm2cVuJ4HYc6fDfsO+kDZOEUCkW5pqIGNhqIr7Mv8enxZdYBKVDg6+zLwEYDzSIhtra2LvG1QqEocax4dZlGowEgMzOT4cOHs2DBglLXquzGpcYgCZAQQlSB4oLVwLqBuNm7sT92P893fF6Snxrk7tFABYoSSVB1GA1s164d69atIyAgACsr808vpAZICCGqwKbLmwB4POhxHm7+MACnkk6ZMiRhAqMCR7F23FrqOdcrcdzX2Ze149ZadEI8Y8YMbt++zaOPPsrRo0eJiopi69atPPnkk6jV9270eP78eU6ePMnt27dJS0vj5MmTnDx50ujxmn+KJoQQFi6nIIftUdsBGNZ0GCk5KQBEJEaYMixhIqMCRzGi2Yhqty+cj48P+/fv57XXXmPgwIHk5eVRv359QkNDUSrvPd4yZMgQrl+/rvs6ODgYwOiNGRVaU7R+NHPp6em4uLiQlpaGs7OzqcMRQli4zZc2M2zVMPyc/bg++zqpuam4fewGwO1Xb1PbvraJIxTllZubS3R0NA0aNMDOzs7U4dRI9/t/oM/7t0yBCSGEkW26VDT9NbzpcBQKBbXtaxPgGgDAycSTpgtMiBrMpAlQQEAACoWi1GPGjBkAfP/99/Tu3RtnZ2cUCgWpqakPvOa7775b6nrNmzc38isRQoiyabVaXf3PsKbDdMeDvYqG+WUaTAjTMGkCdPTo0RL7kWzfXjRHPnbsWACys7MJDQ3ljTfe0Ou6LVu2LHHdffv2GTx2IYQoj5OJJ4lLj6OWdS36NOijOy4JkBCmZdIiaHd39xJff/TRRzRq1IhevXoBMHv2bAB2796t13WtrKz02lMkLy+PvLw83dfp6el63U8IIe6lePprQKMB2Fn9Xa8Q7P1XApQgCZAQpmA2NUD5+fn8+OOPTJkyRddcqaIuX76Mj48PDRs25PHHHycmJua+58+fPx8XFxfdw8/Pr1L3F0KIYhsvbQRgWJNhJY4XjwBdSL5ATkFOlcclRE1nNgnQr7/+SmpqKpMnT67UdTp37szy5csJCwvjm2++ITo6mpCQEDIyMu75nLlz55KWlqZ7xMbGVioGIYQASMxM5OiNowAMbTq0xPd8nHxwd3BHrVVz5uYZU4QnRI1mNn2AlixZwuDBg/Hx8anUdQYPHqz7c+vWrencuTP169dn9erVTJ06tczn2NraYmtrW6n7CiHEP22+tBmAjj4d8XIsOS2vUCgI9g5mW9Q2IhIi6FSvkylCFKLGMosRoOvXr7Njxw6eeuopg1/b1dWVpk2bcuXKFYNfWwgh7qd4+mt40+Flfl8KoYUwHbNIgJYtW4aHhwdDhw598Ml6yszMJCoqyiw3YhNCVF+5hblsv1q0snV4M0mAhHmbPHkyI0eONHUYVcrkCZBGo2HZsmVMmjSp1OZpiYmJnDx5Ujd6c+bMGd1+IcX69evHl19+qfv65ZdfZs+ePVy7do0DBw7w8MMPo1KpePTRR6vmBQkhBLArehfZBdn4OvvSxrNNmecUrwQ7nXSaQk1hVYYnTEyj1nBt9zXOrDrDtd3X0Kg1RrtXWf327n68++67LFq0iOXLlxsthnvZvXs3I0aMwNvbm1q1atG2bVt++umnKrm3yWuAduzYQUxMDFOmTCn1vW+//Zb33ntP93XPnj2BohGj4mLpqKgokpOTdefExcXx6KOPkpKSgru7Oz169ODQoUOlltwLIYQx3b36614rWxu7NcbRxpHM/EwuJF+glUerqgxRmEjk+kjCZoWRHvd3yxVnX2dCF4USOCrQ4PdLSEjQ/fmXX37h7bff5uLFi7pjjo6OODo6Gvy+5XHgwAFat27Na6+9hqenJ5s2bWLixIm4uLgwbNiwB1+gEmQvsDLIXmBCiMrQarXUX1if2PRYNj+2mSFNhtzz3JBlIeyL2cd/R/6XJ9o8UYVRioqo7F5gkesjWT1mNfzznfevHHnc2nFGSYKKLV++nNmzZ5faWWHy5Mmkpqby66+/AtC7d2+CgoJQqVSsWLECGxsbPvzwQx577DGef/551q5di6enJ1988UWJxUdnz57llVdeITw8nFq1ajFw4EA+++wz6tatW+4Yhw4diqenJ0uXLi3z+7IXmBBCmKnTSaeJTY/F3sqePgF97nuu1AFZPq1WS35W/gMfuem5bJm5pXTyA7pjW2ZtITc9t1zXM/b4xYoVK6hbty5HjhzhhRdeYPr06YwdO5Zu3bpx4sQJBg4cyBNPPEF2djYAqamp9O3bl+DgYI4dO0ZYWBhJSUmMGzdOr/umpaXh5uZmjJdUgsmnwIQQorq5u/uzvbX9fc+VBMjyFWQXMN9xfuUvpIWMuAwWuCwo1+lzM+diU8um8ve9hzZt2vDmm28W3WvuXD766CPq1q3LtGnTAHj77bf55ptvOH36NF26dOHLL78kODiYefPm6a6xdOlS/Pz8uHTpEk2bNn3gPVevXs3Ro0f57rvvjPOi7iIJkBBCGNi9uj+XpbgQ+mTiSbRabaU74QthKK1bt9b9WaVSUadOHYKCgnTHPD09Abh58yYAp06dYteuXWXWE0VFRT0wAdq1axdPPvkkixcvpmXLloZ4CfclCZAQQhhQUmYSR+KPACV3f7+XFu4tsFZak5qbyrXUazSo3cDYIQoDs3awZm7m3Aeed33vdVYOWfnA8x774zHq96xfrvsak7V1yesrFIoSx4qTdY2maAVbZmYmw4cPZ8GC0iNYD2pFs2fPHoYPH85nn33GxIkTKxt6uUgCJIQQBvTH5T/QoqWDTwe8nR7cf8xGZUMrj1ZEJEYQkRghCZAFUigU5ZqKajSwEc6+zqTHp5ddB6QoWg3WaGAjlCrLK9Ft164d69atIyAgoFRbm/vZvXs3w4YNY8GCBTz99NNGjLAky/sbFkIIM6bP9FcxXR2Q7AxfrSlVSkIXhRZ98c+Zzr++Dl0YapHJD8CMGTO4ffs2jz76KEePHiUqKoqtW7fy5JNPolary3zOrl27GDp0KDNnzmT06NEkJiaSmJhYot+fsVjm37IQQpih3MJctkVtA+7d/bksxXVAUghd/QWOCmTc2nE41yu5RNvZ19noS+CNzcfHh/3796NWqxk4cCBBQUHMnj0bV1dXlMqy040VK1aQnZ3N/Pnz8fb21j1GjRpl9HilD1AZpA+QEKIitl7ZSuhPofg4+RD3Yly5C5r3x+ynx7Ie+Dj5ED8n3shRisqobB+gYhq1hpjwGDISMnDydsI/xN9iR36qmqH6AEkNkBBCGEh5uj+XpY1XGxQouJFxg6TMJDwdPY0VojATSpWSgN4Bpg6jRpN0UwghDECr1er6/+gz/QXgaONIkzpNAJkGE6KqSAIkhBAGcPbmWa6nXcfOyo6+Dfrq/XwphBaiakkCJIQQBlA8/dW/YX8crB30fn4773aAjAAJUVUkARJCCAPQTX811W/6q5hsiWFZZP2Q6Rjq714SICGEqKSbWTc5FHcIgKFNhlboGsVL4a/cvkJ6XrrBYhOGVdwJuXgDUFH1iv/u/9mpWl+yCkwIISqpuPtzO+921HOuV6Fr1HWoi6+zL3HpcZxKPEVI/RADRykMQaVS4erqqtv/ysHBQfZvqyJarZbs7Gxu3ryJq6srKpWqUteTBEgIISqpstNfxYK9golLjyMiMUISIDPm5eUF/L0JqKharq6uuv8HlSEJkBBCVEJeYR5bo7YC5dv89H6CvYLZeGmj1AGZOYVCgbe3Nx4eHhQUFJg6nBrF2tq60iM/xSQBEkKISthzfQ+Z+Zl4O3rrVnJVlG5LDFkKbxFUKpXB3oxF1ZMiaCGEqITi6a+hTYaiVFTuV2rxSrBzt86RV5hX6diEEPcmCZAQQlSQVqvV9f/Rt/tzWfxd/KltV5tCTSHnbp2r9PWEEPcmCZAQQlTQ+VvnuZZ6DTsrO/o37F/p6ykUCpkGE6KKSAIkhBAVVDz607dB3wp1fy6LNEQUompIAiSEEBWkm/6q5PL3uxUnQCcSThjsmkKI0iQBEkKICkjOTuZg7EGg8svf71Y8BXYq6RRqjdpg1xVClCQJkBBCVEBx9+e2Xm3xdfY12HWb1WmGvZU92QXZXL592WDXFUKUJAmQEEJUgDGmvwBUShVtvNoAUghdHak1anZf282qM6vYfW23jPKZkCRAQgihp3x1PluvGKb7c1mkELp6Wh+5noBFAfRZ0YfH1j9GnxV9CFgUwPrI9aYOrUaSBEgIIfS09/peMvIz8HL0ooNPB4NfXxKg6md95HrGrB5DXHpciePx6fGMWT1GkiATkARICCH0ZMjuz2W5uxeQVqs1+PVF1VJr1MwKm4WW0v8vi4/NDpst02FVTBIgIYTQw93dn40x/QXQyqMVKoWKlJyUUiMGwvKEx4Tf9/+jFi2x6bGEx4RXYVRCEiAhhNBDZHIkV+9cxVZly4CGA4xyDzsrO1q4twBkGqw6SMhIKNd5Xxz5gvDr4eSr840ckQBJgIQQQi/F0199G/Sllk0to91HtsSoPrydvMt13vrI9fRc3pPaC2oT+mMo/97/b47fOC5TY0Zi0gQoICAAhUJR6jFjxgwAvv/+e3r37o2zszMKhYLU1NRyXferr74iICAAOzs7OnfuzJEjR4z4KoQQNYmxp7+KSSF09RHiH4Kvsy8KFGV+X4GC2na1GdtiLO4O7mQXZLM1aiuv7niVDos74P5vd0b9MoqvjnxF5K1IqQszECtT3vzo0aOo1X9ntmfPnmXAgAGMHTsWgOzsbEJDQwkNDWXu3LnluuYvv/zCnDlz+Pbbb+ncuTMLFy5k0KBBXLx4EQ8PD6O8DiFEzZCSncKB2AOAJECi/FRKFYtCFzFm9ZhS3ytOin546AdGBY5Cq9Vy9uZZ/oz+k53RO9lzfQ93cu+w4cIGNlzYAICPkw99G/Slb0Bf+jXsh7+Lf5W+nupCoTWjVHL27Nls2rSJy5cvo1D8nSnv3r2bPn36cOfOHVxdXe97jc6dO9OxY0e+/PJLADQaDX5+frzwwgu8/vrr5YojPT0dFxcX0tLScHZ2rvDrEUJULz+e/pEnNjxBa8/WnHr2lFHvlZabhusCVwCSX0mmjkMdo95PGN/6yPU8suYRCrWFumN+zn4sDF3IqMBRZT6nUFPI8RvH2Rm9kz+j/2R/7H5yC3NLnNOodiP6NehXlBQ16It7LXejvg5zps/7t0lHgO6Wn5/Pjz/+yJw5c0okP/pe4/jx4yVGi5RKJf379+fgwYP3fF5eXh55eXm6r9PT0yt0fyFE9Was7s9lcbFzoWHthly9c5WIxAj6N+xv9HsK4wrxD9ElP4uHLaZxncaE+IegUqru+RwrpRWdfTvT2bczb4S8QW5hLgdjD+oSoiPxR4i6E0XUnSi+P/E9AK09W+tGh3rW74mzrXyQL4vZJEC//vorqampTJ48ucLXSE5ORq1W4+npWeK4p6cnFy5cuOfz5s+fz3vvvVfh+wpRXhq1hpjwGDISMnDydsI/xB+lStYiWIICdQFhV8KAqkmAANp5tytKgBIkAaoODsYVfRBv4d6Cp9o/VaFr2FnZ0adBH/o06ANAel46e6/v1U2ZnU46rXssPLwQlUJFx3oddQlRN79u2FnZGew1WTK9EqDIyEh+/vlnwsPDuX79OtnZ2bi7uxMcHMygQYMYPXo0tra2FQpkyZIlDB48GB8fnwo9vzLmzp3LnDlzdF+np6fj5+dX5XGI6i1yfSRhs8JIj/t7hNHZ15nQRaEEjgo0YWSiPMJjwknPS8ejlgcd63WsknsGewWz9vxaqQOqJorrx7r5djPYNZ1tnRnWdJiuJu1W1i12XdulS4iu3L7CobhDHIo7xLx987BV2dLdv7suIerg0wErpdmMhVSpcr3qEydO8Oqrr7Jv3z66d+9O586defjhh7G3t+f27ducPXuWf/3rX7zwwgu8+uqrzJ49W69E6Pr16+zYsYP16yvXCrxu3bqoVCqSkpJKHE9KSsLLy+uez7O1ta1w4iZEeUSuj2T1mNX8sxFsenw6q8esZtzacZIEmbmNF4umv4zV/bksUghdvegSID/DJUD/5F7LnXEtxzGu5TgAYtJidMnQzqs7SchM4M/oP/kz+k/e3PUmTjZO9AropashauXRqsr+fZtauRKg0aNH88orr7B27dr7FiEfPHiQRYsW8cknn/DGG2+UO4hly5bh4eHB0KFDy/2cstjY2NC+fXt27tzJyJEjgaIi6J07d/L8889X6tpCVJRGrSFsVlip5AcoOqaAsNlhNBvRTKbDzNTd3Z+ravoL/u4FdDH5Iln5WUbtOySMK1+dz9EbRwHjJkD/5O/iz+S2k5ncdjJarZaLKRd1CdGu6F3cyb3DpkubdP2t3B3c6dOgD/0a9KNfg340rN2wwnW55q5cCdClS5ewtrZ+4Hldu3ala9euFBQUlDsAjUbDsmXLmDRpElZWJcNJTEwkMTGRK1euAHDmzBmcnJzw9/fHzc0NgH79+vHwww/rEpw5c+YwadIkOnToQKdOnVi4cCFZWVk8+eST5Y5JCEOKCY8pMe1VihbSY9OJCY8hoHdAlcUlyu9iykWi7kRho7JhQCPjdH8ui5ejF16OXiRmJnI66TRd/bpW2b2FYZ1KPEVuYS5u9m40rdPUJDEoFAqa121O87rNea7jc6g1ak4lnWLn1Z38ee1P9l7fy63sW6w+t5rV51YDRQnU3SvMfJyqvkzFWMqVAJUn+ano+Tt27CAmJoYpU6aU+t63335boji5Z8+eQNGIUXGxdFRUFMnJybpzHnnkEW7dusXbb79NYmIibdu2JSwsrFRhtBBVJSMhw6DniapXPP3VJ6APjjaOVXrvYK9gtlzZQkRihCRAFuzu6S9zGVFRKVW0825HO+92vNL9FfLV+RyJP8LOqzvZGb2TQ3GHiEmLYdnJZSw7uQyA5nWb60aHegX0ws3ezcSvouLKlQB9/vnn5b7gzJkz9Qpg4MCB9+xq+e677/Luu+/e9/nXrl0rdez555+XKS9hNqwdyveBwMnbyciRiIradLloesDYzQ/LokuAZEsMi3YgzvAF0IZmo7Khh38Pevj34J3e75CVn8W+mH26KbMTCSe4kHyBC8kX+OroVyhQ0M67HX0b9KVfg3708O9RrmlatUZNeEw4CRkJeDt5P7AVgLGUKwH67LPPSnx969YtsrOzdfVAqampODg44OHhoXcCJER1Fn8kni0vbLn/SYqi1WD+IdLN1RzdzrnN/pj9QNXW/xTT7QkmhdAWrSoKoA2tlk0tBjUexKDGgwC4k3OH3dd263oQRSZHcjzhOMcTjvPvA//GWmlNF98uuimzzr6dsVHZlLjm+sj1zAqbRVx6nO6Yr7Mvi0IX3bMZpLGUKwGKjo7W/XnlypV8/fXXLFmyhGbNmgFw8eJFpk2bxjPPPGOcKIWwMFqtlmPfHCNsdhiaAg2OXo5kJmX+9c3S54cuDJUCaDMVdiUMtVZNkEcQ9V3rV/n9i1eCnbl5hgJ1AdYq/UoShOnFpsUSlx6n68ljqWrb1+bhwId5OPBhAG5k3GBX9K6iFWbRO4lJiyE8JpzwmHDe3fMuDtYOhPiH6BKi6DvRjFs7Du0/fgnGp8czZvUY1o5bW6VJkN5bYTRq1Ii1a9cSHBxc4vjx48cZM2ZMiWTJUslWGKIy8jPz2fTMJs6sPANA4KhAHlr6ENE7o0v1AQLo+XZP+rzXxxShinJ4dN2j/Hz2Z+b2mMu8fvOq/P4arYbaC2qTnpfOqWdP0dqzdZXHICrnl7O/MH7deDr4dODotKOmDscotFotV+9c1Y0O/Rn9J7eyb5U4R4GiVPJz9/d8nX2JnhVdqekwo26FkZCQQGFhYanjarW6VP8dIWqaW5G3WDNmDbfO30KhUjDg4wF0ebELCoWCwFGBNBvRTNcJ+vza81xYf4Fzv5yj5796orKp+jlwcX8F6gK2XC6awjTF9BeAUqGkrVdb9l7fS0RChCRAFqh4+qurb/UtYlcoFDRya0Qjt0Y83f5pNFoN526e0yVEO67uIKcw557P16IlNj2W8Jhwegf0rpKY9R5z79evH8888wwnTpzQHTt+/DjTp0+nf39p1S5qrrO/nGVxx8XcOn8LR29HJu+eTNc5XUus+FCqlAT0DiDo0SBGLBlBLY9apFxM4dDCQyaMXNzL/tj9pOWl4e7gTqd6nUwWhzREtGy6AmgLqv+pLKVCSZBnELO7zOb3R3/n++Hfl+t5CRkJRo7sb3onQEuXLsXLy4sOHTroOih36tQJT09PfvjhB2PEKIRZU+er2TJzC+vGr6Mgq4CAPgE8E/EM/j3uX9Rs52pH/4+LPjTseX/P/XsFCZMoXv4+pMkQk6xSKVacAJ1IOPGAM4W5ycrP0q3gq0kJ0D/5OvuW6zxvJ28jR/I3vafA3N3d+eOPP7h06ZJug9HmzZvTtKlpGjsJYUppsWmsGbuG+MPxAPSY24M+7/dBaVW+zxZtnmjDie9PEHsglm0vb2PMz2OMGa7Qkym6P5elnXc7AE4mnkSj1dSYrQqqg2M3jqHWqqnnVA8/55q7x2SIfwi+zr7Ep8eXWQdUXAMU4h9SZTFV+KcoICCAZs2aMWTIEEl+RI0UtS2K74K/I/5wPHaudoz/fTz95vUrd/IDoFAqGPLVEBRKBed+OUf0LstfRFBdXEq5xOXbl7FWWldp9+eyNK/bHFuVLRn5GVy9c9WksQj9mGMDRFNQKVUsCl0EFCU7dyv+emHowiodadU7AcrOzmbq1Kk4ODjQsmVLYmJiAHjhhRf46KOPDB6gEOZGq9Gy5/09/Bj6IzkpOXi38+bp40/TbHizCl3Pq60XHaZ3AGDL81tQF6gNGa6ooOLpr94BvXG2Ne1qUGuVNUGeQQDSENHC1MT6n3sZFTiKtePWUs+5Xonjvs6+Vb4EHiqQAM2dO5dTp06xe/du7OzsdMf79+/PL7/8YtDghDA32cnZ/DTkJ3a/sxu00G5aO6bsn0LthrUrdd0+H/TBwd2BW+dvcfjzw4YJVlRKcfdnU09/FZNCaMuj1Wo5GHsQkASo2KjAUVybdY1dk3axctRKdk3aRfSs6CpPfqACNUC//vorv/zyC126dCkxnNeyZUuioqIMGpwQ5iT+SDyrx6wmPTYdK3srhn4zlLaT2hrk2va17en/UX9+n/o7e97dQ9CjQTj5yNYYpnIn5w7h18MB02x/URZJgCzP5duXSclJwc7KjrZebU0djtlQKVVVttT9fvQeAbp16xYeHh6ljmdlZdXo+U1RfWm1Wo58dYSlPZaSHpuOWxM3njr0lMGSn2JtJ7elXud65Gfms/2V7Qa9ttBPcffnlu4taVC7ganDAe7aEkOmwCxGcf1PR5+OpbaEEKandwLUoUMHNm/erPu6OOn54Ycf6Nq1+jZ5EjVTfmY+6x9fz5bnt6Ap0BA4KpBpR6fh2drT4PcqLohGAWdWnuH63usGv4coH3Ob/gJo7dkapUJJUlZSlfZKERVnift/1SR6T4HNmzePwYMHc/78eQoLC1m0aBHnz5/nwIED7NmzxxgxCmEStyJvsXr0apIjk0t1dTYWn/Y+tH+6Pce/O84fz//BMyee0WtVmai8Qk2hrvuzuUx/AThYO9CsTjMikyOJSIyo0n4pomIkATJvev9m7dGjBydPnqSwsJCgoCC2bduGh4cHBw8epH379saIUYgqd/bnoq7OyZHJOPk4ldnV2Vj6/l9f7N3suXnmJke+OmL0+4mS9sfs507uHerY16GLbxdTh1OCTINZjtTcVM7dOgdU7y0wLJneI0BQtCHq4sWLDR2LECanzlez9aWtHP2yaMPCBn0bMGrlKBw9HassBoc6DvSb349Nz2xi99u7afVIKxy9qu7+Nd2mS0XTX6bu/lyWYK9gVp5ZKYXQFuBQXNH2No3dGuNey93E0Yiy6D0C1LdvX957771Sx+/cuUPfvn0NEpQQppAWm8aynst0yU+PN3owYduEKk1+igVPDcangw956XnseG1Hld+/JjOX7s9lkZVglkOmv8yf3gnQ7t27+fLLLxk5ciRZWVm64/n5+VIDJCzWP7s6P7rxUfr9Xz+UKtPU3yhVSl1B9Kn/niJmf4xJ4qhpLqdc5mLKRayUVgxqPMjU4ZRSPAV29c5VUnNTTRuMuC9dAuQrCZC5qtBv9x07dpCYmEiXLl24du2agUMSoupoNVp2v7e7ZFfnE0/TdJjpt3ep16kewVOL3vD+mPEHmkKNiSOq/oqnv3rV72Xy7s9lcbN3o75LfaBoXzBhngo1hRyOL2poKiNA5qtCCZC3tzd79uwhKCiIjh07snv3bgOHJYTxFXd13vPunqKuzk//1dW5QeW6OhtS//n9sattR9KpJI59e8zU4VR75jz9VUwKoc3f2ZtnyczPxNnWmRbuLUwdjrgHvROg4lUwtra2rFy5klmzZhEaGsrXX39t8OCEMJa4w3F81+47orZGYWVvxcgVIxn+3XCs7Cq0LsBoHOo60Pf/imrr/nzzT7JuZj3gGaKiUnNTCY8xr+7PZZE6IPNXPP3VxbeL2RXSi7/pnQBptSW3sX/zzTf56aef+OSTTwwWlBDGotVqOfLlEZaFLPu7q/Php2gzsY2pQ7un9k+3x7udN3lpeex4XQqijWXrla0UagoJrBtII7dGpg7nniQBMn8H4/7a/0vqf8ya3glQdHQ0devWLXFs9OjRHDp0iKVLlxosMCEMLT8zn/WPrWfLC391dR4dyNPHnsYzyPBdnQ1JqVIy+MvBAJxcdpK4Q3Emjqh6soTpL/h7CizyViQ5BTkmjkaURVaAWQa9E6D69eujVJZ+WqtWrZg0aZJBghLC0G5F3mJxp8Wc/fksSislAz8dyNg1Y7F1tjV1aOXi19WPtpPbAn8VRKulINqQCjWFbLlift2fy1LPqR51Heqi1qo5e/OsqcMR/5CYmcjVO1dRoKCzb2dThyPuo1wFD6NGjWL58uU4OzszatT9t6xfv369QQITwlDOrDrDxmkbKcgqwMnHiTGrx+Df3d/UYemt/4L+RG6IJOFEAicWn6DDsx1MHVK1cTD2ILdzbuNm70ZXP/Pu2qtQKAj2Cmb71e1EJEbQsV5HU4ck7nIwtmj6K8gzyCxXEoq/lWsEyMXFRVf87OLict+HEOaiMK+QP57/g/WPracgq4AGfRvwTMQzFpn8ANTyqEWfD/oAsPONnWQnZ5s4ouqjePprSJMhWCnNqxC+LLo6IFkJZnak/4/lKNdP+rJly8r8sxDmKi0mjTVj1xB/JB6AkH+F0Pu93iZrbGgoHad3JOKHCJJOJ7HzjZ0M/96861UsRXH/n2FNzHv6q5huKbwUQpudA3FFCZC5jySKCvYBEsKcXdl6he/afUf8kXjsatvx6KZH6fthX4tPfgCUVn91iAZO/HCC+KPxJo7I8kXdjiIyORIrpRWhjUNNHU65FI8AnU46jVqjNnE0olheYR7HbhT165ICaPNXrhGg4ODgcu+CfeLEiUoFJERFadQa9n6wlz3vFzU29G7nzdi1Y82qsaEh+Pfwp/UTrTn9v9P8MeMPnjr0FAql8Xepr66KR3961u+Ji51lTOM3qdOEWta1yCrI4mLKRWm2ZyZOJJwgX52Pu4M7jWqbbysFUaRcCdDIkSONHIYQlZOdnM36x9cTtS0KgPbPtCd0YajZNTY0lAEfD+Dibxe5cfQGJ5acoP209qYOyWIV1/9YyvQXgFKhpI1XGw7EHiAiIUISIDNx9/L38g4aCNMp17vDO++8Y+w4hKiwuMNxrBm7hvTYdKzsrRj23TDaPGG+jQ0NwdHLkd7v9Wbri1vZOXcnLUa3wN7N3tRhWZy03DT2XC/axHl4M8uqp2rn1Y4DsQc4kXCCx1s/bupwBH/X/8j0l2Ww/KIIUWNptVoOf3G4dFfnap78FOv0fCc8WnmQk5LDn2/+aepwLNK2qG0UagppVqcZjd0amzocvUghtHnRarXSANHC6J0AqdVq/vOf/9CpUye8vLxwc3Mr8RCiKhR3dQ6bGYamQEOLMS0soquzISmt/u4QfezbYyScSDBxRJbHUro/l+XuLTH+uUWRqHrX066TmJmItdKa9t4yJW0J9E6A3nvvPT799FMeeeQR0tLSmDNnDqNGjUKpVPLuu+/qda2AgAAUCkWpx4wZMwDIzc1lxowZ1KlTB0dHR0aPHk1SUtJ9rzl58uRS1wsNtYyVHaJ8bp2/xeKOf3d1HvTZIMasHmMxXZ0NKaBXAK0ebQXaog7RWo28EZaXWqPmj8t/AJY3/QXQ0qMl1kprUnNTuZ523dTh1HjFoz/tvNthby3T0ZZA7wTop59+YvHixbz00ktYWVnx6KOP8sMPP/D2229z6NAhva519OhREhISdI/t27cDMHbsWABefPFFNm7cyJo1a9izZw83btx4YCdqgNDQ0BLXXbVqlb4vU5ipM6vOsLjTYpIvJOPk48Sk3ZPoMrtLjS44HPifgdg42hB3KI6TK06aOhyLcSjuECk5KdS2q22RUxY2KhtaerQEpCGiOZDpL8ujdwKUmJhIUFAQAI6OjqSlpQEwbNgwNm/erNe13N3d8fLy0j02bdpEo0aN6NWrF2lpaSxZsoRPP/2Uvn370r59e5YtW8aBAwcemGjZ2tqWuG7t2vdfBp2Xl0d6enqJhzAvpbo697Psrs6G5OTjRK93egGw47Ud5NyRDTLLo3j6a3CTwRbR/bkssjO8+ZAEyPLonQD5+vqSkFBUa9CoUSO2bdsGFI3m2NpWfAoiPz+fH3/8kSlTpqBQKDh+/DgFBQX0799fd07z5s3x9/fn4MGD973W7t278fDwoFmzZkyfPp2UlJT7nj9//vwS23n4+flV+HUIw0uLSWN5z+Uc/eooACFvhjBh6wRqedQycWTmo/OsztQNrEv2rWx2vb3L1OFYhOL+P5ZY/1NMEiDzkJmfyamkU4AkQJZE7wTo4YcfZufOnQC88MILvPXWWzRp0oSJEycyZcqUCgfy66+/kpqayuTJk4GikSYbGxtcXV1LnOfp6UliYuI9rxMaGsp///tfdu7cyYIFC9izZw+DBw9Grb53t9S5c+eSlpame8TGxlb4dQjD+mdX58c2P0bfD6pHV2dDUlmrGPJlUYfoY18fI/HkvX9GBETfiebcrXOoFCoGNRpk6nAqTLcSTKbATOpI/BE0Wg3+Lv74OPmYOhxRTnqP+3700Ue6Pz/yyCO6EZkmTZowfHjFP0ktWbKEwYMH4+NTuX8848eP1/05KCiI1q1b06hRI3bv3k2/fv3KfI6trW2lRq+E4WnUGva8v4e9H+wt6urc3ptxa8fhGuBq6tDMVoO+DWg5riXnVp/jj+f/4MnwJ2t0bdT9FE9/hdQPoba95XYKb+PZBgUK4jPiuZV1C/da7qYOqUaS6S/LVOmP0V27dmXOnDmVSn6uX7/Ojh07eOqpp3THvLy8yM/PJzU1tcS5SUlJeHl5lfvaDRs2pG7duly5cqXC8YmqlZ2czcohK9n7flHy0/7Z9kzZN0WSn3IY+MlArGtZE7s/ltP/O23qcMyWpW1+ei9Otk66/kUyDWY6sgO8ZapQ5d+NGzfYt28fN2/eRKPRlPjezJkz9b7esmXL8PDwYOjQobpj7du3x9ramp07dzJ69GgALl68SExMDF27ln+X3bi4OFJSUvD29tY7LlH14g791dU5ruZ0dTYkZ19ner7Vk52v72T7q9tpNqIZdi52pg7LrKTnpbP72m7AMpe//1OwdzCXb18mIiGCgY0GmjqcGkej1XAwrqguVUaALIveCdDy5ct55plnsLGxoU6dOiWG2BUKhd4JkEajYdmyZUyaNAkrq7/DcXFxYerUqcyZMwc3NzecnZ154YUX6Nq1K126dNGd17x5c+bPn8/DDz9MZmYm7733HqNHj8bLy4uoqCheffVVGjduzKBBljvPXxNotVqOfHmEbS9tQ1OgoU7TOoxbNw6PVh6mDs3idH2xKyeXnSTlYgq7391N6GfSB+tu26K2UaApoGmdpjSt09TU4VRasFcwq8+tlhEgE7mQfIHU3FQcrB1o7dna1OEIPeidAL311lu8/fbbzJ07F6Wy8oWoO3bsICYmpswC6s8++wylUsno0aPJy8tj0KBBfP311yXOuXjxom4pvkql4vTp06xYsYLU1FR8fHwYOHAgH3zwgdT4mLG8jDw2TtvIuV/OAdBiTAseWvJQjWxsaAgqGxWDPx/Mj4N+5MgXRwieElyjOmQ/SHWZ/iomK8FMq3j6q1O9TlirrE0cjdCH3glQdnY248ePN0jyAzBw4MB7tnG3s7Pjq6++4quvvrrn8+9+rr29PVu3bjVIXMLwNGoNMeExZCRk4OTthH+IP8kXklkzZg3JF5JRWikZ8J8BdJ7ZWYp3K6nRwEYEjgokcn0kW57fwqTdk+TvlKLuz5svF/Urqw7TX/D3SrBLKZfIyMvAydbJxBHVLFL/Y7n0ToCmTp3KmjVreP31140Rj6imItdHEjYrjPS4v5tM2rvZk5+ZjzpfjVM9J8auHotfN+nBZCiDPhvE5S2Xub73OmdXnSXosSBTh2RyR+KPkJydjKudK939ups6HIPwqOVBPad6xGfEcyrpFD38e5g6pBpF6n8sl94J0Pz58xk2bBhhYWEEBQVhbV1yyO/TTz81WHCieohcH8nqMavhHwN9ObeLOhZ7BHkwccdEaWxoYC7+LoT8K4Rdb+5i28vbaDqsaY2fVixe/h7aOLRaTVcEewcTnxFPREKEJEBVKCU7hQvJFwDo4tvlAWcLc1OhBGjr1q00a9YMoFQRtBB306g1hM0KK5X83C33Ti72dWTzQGPo9nI3Ti0/xe0rt9nz/h4G/qdmrxKy5N3f7yfYK5hNlzZJHVAVOxRXtC1T87rNqeNQx8TRCH3pnQB98sknLF26VNexWYj7iQmPKTHtVZb0uHRiwmMI6B1QNUHVIFa2VoR+HsrKISs5vOgwwVOCcW9RM5vlXUu9xtmbZ1EpVIQ2rl4r46QQ2jSk/sey6V3JbGtrS/fu1WPuXBhfRkKGQc8T+msyuAnNRjRDU6jhj+f/uOeig+quePVXd//uuNm7mSQGjVrDtd3XOLPqDNd2X0Oj1jz4SeVQXAh97uY58tX5BrmmeLADcdIB2pLpnQDNmjWLL774whixiGrIybt8K1LKe56omNCFoVjZWXFt1zXOrT5n6nBMwtTTX5HrI1kUsIgVfVaw/rH1rOizgkUBi4hcH1npa9d3qU9tu9oUaAo4d7Nm/v+tagXqAo7EHwGgq1/5m/MK86F3AnTkyBFWrFhBw4YNGT58OKNGjSrxEOJu/iH+OPs63/sEBTj7OeMf4l91QdVArgGu9JhbVBy77aVt5GfWrFGCjLwMXffnYU2rvv9P8UKAf04Hp8ens3rM6konQQqFgrZebQGZBqsqp5NOk12QjaudK83rNjd1OKIC9E6AXF1dGTVqFL169aJu3bq4uLiUeAhxN6VKycBP71F4+1fNfOjCUNndvQp0f7U7tRvWJiM+g70f7jV1OFVq+9Xt5KvzaezWmGZ1mlXpve+7EOCvY2Gzwyo9HaarA5Kd4atEcf1PV9+uKBXy+8sS6VUEXVhYSJ8+fRg4cKBeG5KKmq0wtxAAhVKBVvP3u4CzrzOhC0MJHBVoqtBqFCs7K0IXhbJq+CoOfnqQtk+2pW6zuqYOq0oU1/8Mbzq8ylerPnAhgBbSYyu/EKC4DkhGgKqG1P9YPr0SICsrK5599lkiIys/Zy1qBo1aQ/iH4QD0+aAPft38SnSClpGfqtV0WFOaDG3C5c2X2fLCFiZsnVDt21dotBpd92dTTH9V1UKA4hGgU0mn0Gg1MiphZLoVYJIAWSy9f0I6depERIR8whDlc271OVIupWDvZk+nFzoR0DuAoEeDCOgdIMmPiYQuCkVlq+Lq9qsGKcA1d0fij3Az6ybOts6E+IdU+f2raiFAs7rNsLOyIzM/kyu3r1TqWuL+4tLjiEmLQalQ0qleJ1OHIypI73eg5557jpdeeokvv/ySgwcPcvr06RIPIYppNVr2flBUa9LlxS7YOtXsLsTmwq2RG91fLWplsfXFreRnVe+C6OLpL1N1f/Zo7YGV3f0H2+3r2ld6IYCV0oo2nm0AOJFwolLXEvd3MLZo+4s2nm1wtHE0cTSiovROgMaPH090dDQzZ86ke/futG3bluDgYN1/hSh2ft15kiOTsXWxpdML8inJnPR4vQcu9V1Ij00nfF64qcMxKlMuf0+5lMLSbkt1dXD3kpOcw9Gvj1b6flIIXTVk/6/qQe8EKDo6utTj6tWruv8KAf8Y/ZndBTsXOxNHJO5m7WBN6MKibsgH/3OQlMspJo7IOK6nXud00mmUCiWDGw+u0ntfCbvC4k6LSbmYgrOvM/0/7l+qJYSzrzMNBjQAIGxmGFtf2lpioYC+pBC6akj9T/Wg91YY9evXN0Ycopq58NsFbp65ia2zLZ1ndTZ1OKIMzUY0o3FoY66EXSFsZhiP/fFYtSuILi5+7ubXrcr2atJqtRz89CA7Xt2BVqPFr5sf49aNw9HLka5zuhITHlNiIYBCqWD/gv3snLuTQ58eIu16Gg//72Gs7fWfrrt7SwytVlvt/n+ag5yCHN0UoyRAlq1CVahRUVG88MIL9O/fn/79+zNz5kyioqIMHZuwUFqtlr3vF43+dJrZCfvastGpOVIoFIR+HorKRsWVsCtc/P2iqUMyuKqe/irMLeTXSb+y/eXtaDVagqcGM/HPiTh6FdWJKFXKUgsBFAoFPV7vwaifRqGyURG5LpL/9vsvWbey9L5/kGcQKoWK5Oxk4jPiDf3yBHA84TgFmgK8Hb2p7yIDApZM7wRo69attGjRgiNHjtC6dWtat27N4cOHadmyJdu3bzdGjMLCXNp0icSTidg42tBldhdThyPuo06TOnR9uaiN/9bZWynIKTBxRIaTmZ/Jn9F/AlWTAGXcyGB5r+Wc/t9pFKqi5HL44uFY2ZZvoD3osSAmbJuAnasdcQfjWNptKbev3NYrBjsrOwLdi/pqSR2QcegaIPp1lRE2C6d3AvT666/z4osvcvjwYT799FM+/fRTDh8+zOzZs3nttdeMEaOwIHeP/nR8viMOdRxMHJF4kJA3QnD2cyb1Wir7Ptpn6nAMZsfVHeSr82lYu6HRtyqIPxLP9x2+J/5IPHa17ZiwdQKdX+is9xtkQK8AphyYgmuAK7ev3OaHLj8QeyBWr2vIzvDGJTvAVx96J0CRkZFMnTq11PEpU6Zw/vx5gwQlLNeVsCvcOHYDawdrus6RDQItgU0tGwZ9NgiA/Qv2cztKv1EHc7Xx4t/TX8b8pH7qf6dY1nMZmQmZuLdwZ9rRaTTs17DC13MPdGfqwan4dPAhJyWHFX1XcH5d+X+3SgJkPFqtVgqgqxG9EyB3d3dOnjxZ6vjJkyfx8PAwREzCQmm1Wva8tweADs91oJZ7LRNHJMorcFQgDfs3RJ2nZuvsraYOp9KqovuzRq1h28vb+HXir6jz1DR7qBlTD07FrZFbpa/t6OXIpN2TaDq8Keo8NWvGruHgZwfRah+8Qky3EkymwAwu6k4Ut7JvYaOyoZ13O1OHIypJ7wRo2rRpPP300yxYsIDw8HDCw8P56KOPeOaZZ5g2bZoxYhQW4uqOq8QfjsfKzopuL8unI0uiUCgY/MVglNZKLm26xKVNl0wdUqUcu3GMpKwknG2d6Vm/p8Gvn5uay6phqzj4SVE/mJA3Q3hkwyPYOhuu2adNLRse2fAIHZ7rAFrYNmcbYbMevGlq8a7w19OuczuneozmmYvi0Z8OPh2wtarY/2uNWsO13dc4s+oM13Zfq/QmuKLi9F4G/9Zbb+Hk5MQnn3zC3LlzAfDx8eHdd99l5syZBg9QWIa7R3/aP9seR0/pjmpp6javS5cXu3Dg4wOEzQqjYf+GD+xgbK6Kp78GNRqEjcrGoNdOvpjMzw/9TMqlFKzsrRi5fCQtx7U06D2KKVVKhnw5hNoNarP9le0c+eIIaTFpjF45GmuHspfJu9q50sC1AdGp0ZxMPEnfBn2NEltNVNn6n8j1kYTNCiuxOa6zrzOhi2RTaFPQewRIoVDw4osvEhcXR1paGmlpacTFxTFr1iypiK/Bru2+Ruz+WFS2Krq/0t3U4YgK6vVWL5zqOXHn6h32f7zf1OFU2KbLRdtfGHr66/Ifl/mh0w+kXErB2c+ZKfunGC35KaZQKOj2cjfGrB6DylbFxd8usrz3cjKTMu/5HJkGM47K1P9Ero9k9ZjVJZIfgPT4dFaPWV0j9uUzN5XajdLJyQknp8pt4Ceqh+KVX+2mtcPJR/5NWCobRxsGfjIQgH3z93En+o6JI9JfbFosJxNPolQoGdJkiEGuqdVq2f/xflYOW0leeh7+PfyZdnQa3sHeBrl+ebQc25KJOydiX8eeG0dvsKTrEpIvJpd5bjuvovoUKYQ2nLTcNM7ePAsULYHXh0atIWxWGJRVwvXXsbDZD57eFIaldwKUlJTEE088gY+PD1ZWVqhUqhIPUfNc33uda7uvobJR0eO1HqYOR1RSy3EtadC3AYW5hWybs83U4eitePPTrr5dqetQt9LXK8gpYMMTG9jx2g7QFiX5E3dONMk0r393f6YemErthrVJjU5lSdclXA+/Xuq84hEg2RTVcI7EH0GLloa1G+Ll6KXXc2PCY0qN/JSghfTYdGLCYyoZpdCH3hP8kydPJiYmhrfeegtvb2+Z9hLseb+o9qftlLal9joSlqe4IPrbNt9y4dcLXAm7QuPQxqYOq9wMOf2VHp/OLyN/4caxG0XNDReF0vG5jib9vVenaR2mHprKquGriD8cz//6/4+R/x1Jq0da6c4pXgp/MeUi2QXZOFhLP67Kqsz0V0ZChkHPE4ahdwK0b98+wsPDadu2rRHCEZYmZn8M0TujUVop6fG6jP5UF+4t3Ok8qzMHPznIlhe2MP3s9HJ3NDalrPwsdl7dCVS++3PcoTh+efgXMhMzsXezZ+yasTTo28AQYVZaLfdaTPpzEusnrOfChgusG7+O1GupdH+1OwqFAm8nbzxreZKUlcTppNN08ZWO7JV1IK4KGiBWfB9cUQF6T4H5+fmVqxeFqBmKd3xvM7kNrvVdTRuMMKhe7/TC0duR21du65Z7m7ud0TvJU+cR4BpAC/cWFb7OyRUnWd5rOZmJmXi08mDa0Wlmk/wUs3awZuyasXSeXbTZ8M7Xd7L5uc1oCovqSKQQ2nDUGjWH4g4B+tX/qAvU7P/3fn6b8lu5zt8waQNbZm2p0D5wQn96J0ALFy7k9ddf59q1a0YIR1iSuMNxRG2NQqFSEDI3xNThCAOzdbJl4H+KCqL3friXtJg0E0f0YJXt/qwp1LB1zlZ+m/wb6nw1zUc2Z8qBKdRuWNvQoRqEUqUk9LNQBi0cBAo4/u1xfh7xM/mZ+dIR2oDO3zpPel46jjaOtPJo9eAnULQ9yuKOi9nx6g7UuWrcW7qDgqLH3f465tnWE22hliOfH+HzRp+z98O95GflG/qliLvonQA98sgj7N69m0aNGuHk5ISbm1uJh6g5dKM/T7Qx2zcIUTmtHm1F/Z71KcwpZOsc8+4QrdFqdPU/FZn+yrmTw09DfuLQZ0Wf9Hu+3ZNx68Zh62S45obG0mVWF8atG4eVnRWX/7jM8l7LaaUqeqOWBKjyiut/OtfrjJXy/lPBeel5bJm5hR+6/EDSqSTs3ex5aOlDTD8znXFrx+Fcr2SdpLOvM+PWjuPZiGd5YscTeLfzJj8jn11v7eKLxl9w7LtjulE9YVh6T+ovXLjQCGEIS3Pj+A0ub76MQqkg5F8y+lNdKRQKBn85mO+CvyNyXSRR26NoNKCRqcMq04mEEyRmJuJo46h39+dbkbf4+aGfuX3lNtYO1oxcMZIWYyo+hWYKgQ8HMmnXJFYNX0XCiQQcnnDA/SF3zqjOUKAuwFpVduNE8WC6+p8HFEBf+PUCfzz/BxnxRcXMrZ9ozcBPBuq2BQocFUizEc2ICY8hIyEDJ28n/EP8UaqKxiIa9mvItKPTOLf6HDvf2ElqdCqbn93Moc8O0W9+P5qPbC4LjwxI7wRo0qRJxohDWJji0Z+gx4Nwaywjf9WZZ5AnnZ7vxOFFh4sKok9PR2Vjfi0v7u7+rM82BZc2XWLdY+vIz8jHxd+F8b+Nx6utfsuczYVvF1+mHprKT4N/4vbl20xdOpWfH/mZC8kXCPIMMnV4FutBK8DS49L54/k/uPjbRQBqN6rNsG+H0bB/6U1xlSolAb0D7nkvhVJBq/GtCBwVyLFvj7H3g72kXExh9ajV+Hb1ZcDHA/Dv4V/5FyXKNwWWlaVfQVZ5zw8ICEChUJR6zJgxA4Dc3FxmzJhBnTp1cHR0ZPTo0SQlJd33mlqtlrfffhtvb2/s7e3p378/ly9f1it+cX+JJxOLftAVyOhPDdH7vd7U8qxFysUUDn5mngXRGy/9Xf9THlqtln0f7WPVQ6vIz8infs/6TDs2zWKTn2JujdyYenAqft39sMu144n/PUH4knBTh2Wxbmbd5MrtKwClVtNp1BoOf36YrwK/4uJvF4tWw77Rg+lnppeZ/OhDZaOi88zOzIyaScibIVg7WBN3MI5lIcv4ecTP3Dp/q1LXF+VMgBo3bsxHH31EQkLCPc/RarVs376dwYMH8/nnn5fr5kePHiUhIUH32L59OwBjx44F4MUXX2Tjxo2sWbOGPXv2cOPGDUaNGnXfa3788cd8/vnnfPvttxw+fJhatWoxaNAgcnNzyxWTeLC9HxaN/rQa34q6zSrfaE6YPzsXOwZ8PAAoGv27b1M3E4hPjyciMQIFCgY3GfzA8wuyC1j/+Hp2zt0J2qL9657Y/oRuqsLSOdRxYOKOieT1yEOlUXHrX7fY++FeWcFbAQdjixL+lu4tcbVz1R1PPJnIkq5LCJsVRn5mPn7d/Hgm4hn6/V8/rO0NN91o62xL3w/68sKVF2j/THsUKgUXf7/IN0Hf8PtTv5Meb14/i5ZEoS3HT8TFixd544032Lx5M23atKFDhw74+PhgZ2fHnTt3OH/+PAcPHsTKyoq5c+fyzDPPVKgr9OzZs9m0aROXL18mPT0dd3d3Vq5cyZgxYwC4cOECgYGBHDx4kC5dSve10Gq1+Pj48NJLL/Hyyy8DkJaWhqenJ8uXL2f8+PHliiM9PR0XFxfS0tJwdpbGfne7efYm3wR9AwqYfmY6Hi09TB2SqCJarZZlIcuI3R9Ly0daMubnMaYOSee7Y9/x7OZn6erblQNTD9z33PS4dH4e+TMJxxNQWikJ/TyUjtM7VlGkVWvZiWWsemEV3Q8U7c8XPDWYod8MRWVtflOY5uq17a/x8YGPmdZuGt8P/578rHx2v7ubQ58dQqvWYutiS/+P+tP+6fYolMavz0m+kMzON3ZyYcMFAKzsregyuwvdX+2Onaud0e9v7vR5/y7XCFCzZs1Yt24dly5dYty4ccTHx7N27VoWL17M7t27qVevHosXL+batWs899xzFUp+8vPz+fHHH5kyZQoKhYLjx49TUFBA//79dec0b94cf39/Dh4sewg+OjqaxMTEEs9xcXGhc+fO93wOQF5eHunp6SUeomzFoz8txrSQ5KeGUSgUDPlqCAqlgnO/nCN6V7SpQ9Ipb/fn2AOxfN/hexKOJ2Bfx54ndjxRbZMfgHY+7dg+cDs7H9qJQqkgYkkEq4avIi89z9ShWYy7C6Avb7nM1y2/5uB/DqJVa2k5riUzImfQ4dkOVZL8ANRtXpdH1j/ClP1T8OvuR2FOIfvm7+PzRp9z8LODFOYVVkkc1YFey+D9/f156aWX+PXXX4mIiODChQvs27ePL774gmHDhlVqL7Bff/2V1NRUJk+eDEBiYiI2Nja4urqWOM/T05PExMQyr1F83NPTs9zPAZg/fz4uLi66h5+fX4VfR3V2K/IW51afA6Dnm/qtshHVg1cbLzo81wGALc9vQV2gNnFEkF2QzY6rO4D71/9ELI1gRZ8VZCVl4dnak6ePPU1Ar4AqitI0Wri3wEZlQ3i7cHr92AtrB2uitkaxLGSZTJ2UQ746n2M3juGY4Yj2PS0rh6wk7XoaLvVdeGzzY4z5ZQxO3qbZ/Nmvmx9Phj/J+N/GUzewLjm3c9g2ZxtfNvuS0z+eRquR6c4HqdRu8Ia0ZMkSBg8ejI+PT5Xfe+7cuaSlpekesbGxVR6DJQj/v3DQQvOHm+PZ2vPBT/gHtUbN7mu7WXVmFbuv7UatMf2bp9Bf3w/64uDuwK3ztzj8+WFTh8POqzvJLcylvkv9MpvUaQo1bJm1hd+n/o46X03g6ECm7J+Ca4Br1QdbxaxV1gR5FK3+utXmFpP3TKaWZy2STiexpMsSkk7ff1FJTRdxI4KWh1rywlcvEPNrDAqlgq4vdeW5c8/RZEgTU4eHQqGg2UPNmH56OsN/GI6TjxNp19PY8MQGvm//PVHbokwdolkziwTo+vXr7Nixg6eeekp3zMvLi/z8fFJTU0ucm5SUhJdX2as0io//c6XY/Z4DYGtri7Ozc4mHKCnlUgpnV50FoOdb+o/+rI9cT8CiAPqs6MNj6x+jz4o+BCwKYH3kekOHKozMztWO/guKppn3vLuHjBum3cCxePf3YU2HleqRknM7h58G/8SRz48A0OvdXoxdPRYbR5sqj9NUdB2hEyLw6eDDU4eeom5gXdLj0lnaYylR2+VNsiw3z91k+7DtDN80HNtcW3w6+DDt2DQG/mcgNrXM69+P0kpJu6nteOHyC/Sd1xdbZ1sSTyby46Af+d+A/3Hj+A1Th2iWzCIBWrZsGR4eHgwdOlR3rH379lhbW7Nz507dsYsXLxITE0PXrmXvxdKgQQO8vLxKPCc9PZ3Dhw/f8zmifMLnhaPVaGk6vCnewd56PXd95HrGrB5DXHpciePx6fGMWT1GkiAL1HZSW+p1rkd+Zj7bX9lusji0Wu09uz/fPHeTxZ0Wc3XHVaxrWTNu3Th6v9O7ymo1zIVuT7C/OkK7BrgyZf8U6veqT35GPiuHrCRimXSLLlaQU8Cfb/7Jd8HfoT6tJs8mD9ULKqYemqr3776qZu1gTcjcEGZGzaTLi11Q2ai4uuMqizssZt1j67hz9Y6pQzQrJk+ANBoNy5YtY9KkSVhZ/d2X0cXFhalTpzJnzhx27drF8ePHefLJJ+natWuJFWDNmzdnw4YNQNFw4OzZs/nwww/5/fffOXPmDBMnTsTHx4eRI0dW9UurNm5H3eb0j6cB/Ud/1Bo1s8JmoS1jm+PiY7PDZst0mIVRKIsKolHAmZVnuLbnmkniOJFwghsZN6hlXYveAb11xy/+fpElXZZwJ+oOrgGuTD0wlcBRgSaJ0dTK2hPMvrY9E7ZOIOixIDSFGn6f8ju73tlV45fJX915lW9bf0v4/4WjKdBwreU1vprxFSFzQnTdmi2BQ10HBn06iOcvPk/Q40VToGdXneXL5l/KZqt30bsTtKHt2LGDmJgYpkyZUup7n332GUqlktGjR5OXl8egQYP4+uuvS5xz8eJF0tL+3qTx1VdfJSsri6effprU1FR69OhBWFgYdnayPLCiwueFo1VraTy4MfU61tPrub9e+LXUyM/dtGiJTY+l3qf1aOTWiHpO9fBx8vn7v85/f13Lpnr0aKkufNr70P6Z9hz/9jhbnt/C0yeervLl1cXTXwMbDcTWyhatVkv4vHB2vbULtBDQO4Cxa8biUNehSuMyJ609W6NAQWJmIomZiXg5FpUDWNla8fD/HsYlwIV98/ax9/29pF1LY/ji4WbZ6duYsm5lsf3l7Zz67ykAnHyc6LigIz2v9ESlVNGxnmWuFHQNcGXUj6Po+lJXdr6+k6htURz5/Agnl52k+2vd6TK7i9lN51WlcvUBultYWBiOjo706NEDgK+++orFixfTokULvvrqK2rXtvxNMaUP0N/uRN/hy6ZfoinUMOXAFPy63n+FnFar5WTiSTZe2sjGSxs5duOYwWJxsXUplRT9M1nycvR64GaFwnBybufwRdMvyEnJYdBng+gyu3R/LmPq8H0HjiccZ+lDS5nQbAK/TfmNc78UrVTs8FwHQheGSs8bIPCrQC4kX+CPx/4os1Hk8cXH2Tx9M1q1lgZ9GzBu3bga0VNGq9VyasUptr28jZyUHFBAxxkd6fd//dgQs4FH1z1KB58OHJ121NShGsTVHVfZ/up2EiOKVkU7ejvS+93eBE8JRmllOSNc96PP+7fe7xSvvPIKCxYsAODMmTO89NJLummqOXPmsGzZsopFLczSvo/2oSnU0HBAw3smPzkFOfwZ/ScbL21k06VNxGfE632fLwd/iaejJzcybhCfHs+NzL/+m3GD+Ix4MvMzSctLIy0vjcjkyHteR4ECT0fPMpOju/9c2662STYVVGvUhMeEk5CRgLeTNyH+IaiUlvsGbe9mT7/5/dj09CZ2v7ObVuNb4ejlWCX3vpFxg+MJx1GgIMQuhKU9lpIYkYjSSsmQr4bQ/un2VRKHJQj2CuZC8gUiEiPKTIDaT2uPs68za8etJfrPaJb2WMrjfzyOi7+LCaKtGimXUtj07Cau7boGgGdrT4Z9Pwzfzr7AXft/+d5/A1RL0rB/Q54+9jRnfznLn//6k9ToVDY9s4mDnx6skZut6p0ARUdH06JF0S7J69atY9iwYcybN48TJ04wZMgQgwcoTCctJo2Ty04C0OvtXiW+l5CRwObLm9l4aSM7ru4guyBb9z0HawcGNBzAsKbDCG0UStelXYlPjy+zDkiBAl9nX57t8Ox9E4H0vPS/k6O/kqLi/xYfS8hMoFBTqBvqP55w/J7Xs7Oyw8fJp9RI0j9Hl+yt7fX8W7u39ZHrmRU2q8SUoK+zL4tCFzEq8P5bvJizdlPbcWLxCW4cvcGO13YwcsXIKrnv5kubARiUPYhfe/9K1s0sHOo6MG7dOOr3rF8lMViKYK9gVp1dVaIO6J+aDG7C5L2TWTl0JbfO3eKHLj/w2ObHzL7wV1+FeYXs/3g/4f8XjjpPjZW9Fb3f7V1UNHzXaOGDNkC1VAqlgqBHg8rcbNWvmx/9P+6Pf/easdmq3lNgbm5u7Nu3jxYtWtCjRw8mTpzI008/zbVr12jRogXZ2dkPvoiZkymwIptnbObY18cI6BPAxJ0T7zu15evsy7AmwxjebDh9AvqUSByKV4EBJZIgBUWfNNaOW2uQBECj1XAr69bfydFdydLdx1JyUsp9zdp2tUtNu/2zPsmzlucDR3GK/w7+mQQa+u/AVOKPxvND5x9AC0/ue7JKfoE+tOoh4lfF89CWh6AQPNt4Mv638bjWdzX6vS3Njqs7GPC/ATSq3YgrM6/c99y0mDR+GvITt87dwsbRhjGrx9BksOl73hjC9fDrbHpmE8mRyQA0GtSIod8MpXaDkqUbWflZuHzkglqrJmZ2DH4u1bc5bm5aLgf+fYCDnx6kMKeoi3Szh5rR76N+uAe6mzg6/enz/q13AvTQQw+Rn59P9+7d+eCDD4iOjqZevXps27aN559/nkuXLlUqeHMgCRCkx6fzecPPUeerSZmXwga7DaWKmTv6dGR40+EMazqMtl5t7zt0Wtboh5+zHwtDF1b5G39uYS4JGQklR5LS40uNKuUU5pTrekqFEi9Hr79HkBxLJkyetTwZvHIwNzLK7sVRPAoWPSvaoqfDNj69kROLT+DZpqjLsjFrCjKzM3ms32O0P1Q0zdVibAtGLBtRows67yclO4W6/y7auDj1tVRc7O4/tZWblsvq0auJ3hmNQqVg6NdDLXpKMedODttf3U7ED0UjYLU8ahG6KJSWj7Qs8/fW7mu76bOiD77OvsS+WDMa42bcyGD3e7uJWBKBVq1FoVTQdkpber/bG+d6lvM+aNQEKCYmhueee47Y2FhmzpzJ1KlTgaKd29Vqdbl3gjdnNTkBKp7aivhXBB5hHlyrf43lTy4HwN7KngGNBjC86XCGNhmKt5N+Q+OWVP+i1WpJy0srPZL0j/qkxMxE1FrDLOHfNWlXiaXcliY7OZsvm31Jzu0cBn8xmE7PdzLOfVKy+Xbot2QcLmrA2Pv93vR8s2eNql2oCP/P/IlNj2XP5D30rP/gdhbqfDUbn97IqRVFK6N6zO1B3w/7WlQfJa1Wy9mfz7J19laybhYt/W43rR39F/THvva9p7fnh8/njT/fYFzLcfwy5peqCtcsJF9IZufcnVz49R+brb7WHTsX8y+MN2oRtL+/P5s2bSp1/LPPPtP3UsIMlLVqyzHDkVk7ZgFwfvB5nm3/bJlTW/pSKVUW8wavUChwtXPF1c6VFu4t7nmeWqPmZtbNe44k3ci4QfSdaLIKHtx3IyEjwZAvoco51HWg7//1ZfP0zfz55p+0HNeSWh6GbV1w8+xNVj20iozoDPJs8sh5JYdeb/V68BMF7bzbEZseS0RCRLkSIJWNihHLRuAa4Mqe9/awb/4+Uq+lMmLZCKxszX+l5Z3oO2yevpmorUWdrusG1mXYd8OoH/Lg+jDdBqjVqAC6vOo2r8sjGx4h9kAs21/dTuz+WPbN38fx74/T882edJjewSL+/5dHhV5FVFQUy5YtIyoqikWLFuHh4cGWLVvw9/enZcuWho5RGFhOQQ67ru1i48WNbLq8qdTU1qjTo7AutKZ2h9oc/OogSmX1WB5pDCqlCm8n7/uOhhUPpz+IviNq5qjdtKKC6IQTCex4fQcjlo4w2LUv/HqBDU9sID8zn3S3dH585EeWT1pusOtXd8Fewfx28bf7FkL/k0KhoPe7vXGp78KmpzdxdtVZMm5k8MiGR+47gmJK6gI1hz47xO53d1OYU4jKVkXIv0Lo/mr3cr1xa7XaalsArY/izVYv/n6Rna/vJPlCMltf3MrhRYfp82Efgh4NsqjRwLLo/c62Z88egoKCOHz4MOvXryczMxOAU6dO8c477xg8QGEYCRkJ/HDiB0b8PIK6/67L0JVD+fb4t8Slx2FvZc9DzR5i8fDFRD0RRdN9TQEY8uEQSX4MIMQ/BF9nX13Bc1n8nP0I8Q+pwqiMQ6lSMvjLomXWJ5edJPZg5esntFotez7Ywy8P/0J+Zj51e9Tlm6e+IdM3kz4NHpxYiiLFW2KcSDih/3OfDObxLY9j42TD9T3XWdptKXeizW9bhbjDcSzusJgdr+2gMKeQgD4BTD89nV5v9Sr3qMWllEvczrmNnZUdbbzaGDli86ZQKGg+ojnTz0xn+OKizVZTr6WyYUL12GxV73e3119/nQ8//JDt27djY/N3wWHfvn05dOiQQYMTFafVaolIiOCDPR/QaXEnfD71YdrGafx+8XeyC7Kp51SPZ9s/y+bHNpPyagq/jf+Np9o9RdR3URTmFFKvUz0aDWxk6pdRLaiUKhaFLgK4ZxLUyqNVtalh8evqR9sn2wKw5fktaNSaCl8rPyuftePWsvvt3QB0eqETKe+mkOOQw4CGA7CzMv+aBHNRvCXG+VvnyS3M1fv5Dfs3ZMr+KTj7OpN8IZklXZZw45h5bLKZl57HH8//wZKuRTvc27vZM2LZCCbunEidpnX0ulbx6E9Hn47YqKSoHv7abPWpe2+2mnDCMqfv9U6Azpw5w8MPP1zquIeHB8nJyQYJSlRMbmEuf1z+g+mbpuO/0J9237fj7d1vc/RGURfTjj4deb/3+5x4+gSxL8byzbBvGNJkiK6uJzs5m6NfFZ3b820pKjWkUYGjWDtuLfWcS24l4mrnCsCWK1uYsH4C+ep8E0RneP0/6o+dqx0JJxI4sVj/EQeA1OupLO2+lPNrz6O0VjJ88XAGfz6YTVfL3vxU3J+vsy917Oug1qo5e/Nsha7hGeTJ1ENT8WzjSdbNLJb3Ws7FjRcNHGn5abVaItdH8lXgV0W/u7TQZmIbZlyYQdvJ91+Zei8y/XVvd2+22nl2Z5TWSq7uuMr37b8v2mzVDEcF70fvGiBXV1cSEhJo0KBBieMRERHUq6ffPlGi8hIzE9l0aRObLm1i+9XtJRoS6rtq6+BnBynIKsC7nTdNhlSPvh/mZFTgKEY0G1FqJdzqc6uZ+OtEVp1dRUpOCuvGrcPRpmq6KRtLLY9a9PmgD1te2MLON3bSYkwLvfbjuh5+ndWjV5N9K5taHrUYt24c/j38SchI0CX0Q5sONVb41ZJCoSDYO5gdV3cQkRBBB58OFbqOcz1nntz7JGvGrSFqaxS/jPyF0M9D6TTDOKv+7iUtNo0tz2/h4u9FCZhbYzeGfjuUhv0aVuq6ugJoSYDuyaGuA6GfhdJ5Zmd2vbWLMz+d4eyqs5xfe56Oz3Uk5F8h1HI3/70b9U6Axo8fz2uvvcaaNWtQKBRoNBr279/Pyy+/zMSJE40Ro7iLVqvlVNIpNl4sWrVV/GZQrJ5TPYY3Ha73qq2c2zkc+eIIIKM/xlTWSrhHgx7Fzd6N0atHsy1qG/3+24/Nj22mrkNd0wRpIB2e7cCJH06QdCqJnW/sZPj35RuxOfbdsaKps0INXsFejP91vG5Lhj8u/wEUjWYWb+opyi/Y668ESI9C6LLYOtvy6MZH2Tx9MxFLItjy/BZSr6UyYMEAoxfGatQajnx5hF1v7iI/Mx+llZLur3Un5F8hWNtbV+rad3LucP7WeQC6+nY1RLjVWu0GtUtttnp40WEilkZYxGareidA8+bNY8aMGfj5+aFWq2nRogVqtZrHHnuMN9980xgx1ni5hblFe23dY9VWR5+ODGs6jOFNhz+wIeG9HFp0iPyMfDzbeNLsoWaGCl2U06DGg9g5cSdDVw7lSPwReiztwbYntuHvYrkt6Yv35FrWYxknfjhBu2ntqNfx3qPE6gI1YbPCOPZNUZfxlo+0ZMTSEVg7/P2mtvHSRkCmvyqquA6osgkQgMpaxfDFw3Ft4MquN3dx8D8HSbuexsP/fRgrO+Msk06ISGDjtI0kHC+qOfHr5sew74fh0dLDINc/FFdUx9rErQnutSyvC7KpeAd7M2HrhBKbre56cxdHvzpK7/d6E/xkyc1WNWoNMeExZCRk4OTthH+IP0pV1S+40bsRYrHY2FjOnDlDZmYmwcHBNGlSfaZMjNUIUZ9GgImZiWy+VLTXVmWnth4kNzWXhQELyUvLY+zasbQYfe++N8K4Im9FMujHQcSmx+Lr7MvWCVvv24fIEvw66VdO/fcUPh19eOrQU2WOEGQnZ7Nm7Bqu7b4GCuj7YV96zO1RIpnPLcylzsd1yC7IJuKZCNp6ta26F1FNXEi+QOBXgThYO5D+errBGpGe/vE0v035DU2BBr/ufoz/bTwOdco/5fkg+Zn57HpnF4cXHkar0WLrYkv/Bf1pP629QUec3vrzLT4M/5BJbSaxfORyg123JtFqippP/vmvP0m9lgoU9RbqN78fzUY048KGC4TNCiM9Ll33HGdfZ0IXhRI4KrDS9zdqJ+j333+fl19+GQeHkv+4c3Jy+Pe//83bb7+tf8RmxhgJ0IM2wjTW1FZ57Hl/D7vf2Y17S3emn55u8b0dLF1sWiyDfhxEZHIkbvZubH5sM118u5g6rArLTMrky6Zfkpeex9Bvh1K3Wd0Sn/xunbvFzyN+JvVaKjaONoxaOYpmw0uPQm65vIUhK4fg6+xLzOwYmaatALVGjfNHzmQXZHP+ufMEulf+DadY9K5ofnn4F/LS8nBr4sbjWx7HrZFbpa97afMl/njuD9Ji0gBoOa4lgxYOwsnbqdLX/qd+/+3Hn9F/8t2w73i6/dMGv35NUphXqNtsNSelaFuhOs3qkHKxjL0Y//pRHrd2XKWTIKMmQCqVioSEBDw8Sg45pqSk4OHhgVptmG0BTMnQCdD9NsLUomVQo0Gcv3We2PSSPVMMMbX1IHnpeSysv5Dc1FxG/zyaVo+0Mvg9hP5SslMYunIoh+MP42DtwLpx6whtHGrqsCrs0KJDbJ29tWjd6V2r4u3r2JOfmY86T03tRrUZ/9v4e05nPLf5Ob459g3Ptn+Wb4Z9UzWBV0PdlnTjYNxBfhr1E48FPWbQa988d5OVQ1aSFpOGQ10HHt34KL5dfCt0rYyEDMJmhXF+TVFNjkt9F4Z+PdRoCzQKNYW4fuRKVkEWZ6afoZWH/C40hOLNVg98cgB17n3yA0XRSNCs6FmVmg7T5/1b77totdoy34hPnTqFm1vls/3qRq1RMytsVqnkB/7eGX1r1FZi02NLNCS8MecGR6Yd4e1ebxPsHWy0T7tHvjxCbmoudZvXpcUYy55qqU7qONRh58SdhDYOJbsgm+GrhvPT6Z9MHVaFOfn89Wn9Hy2BclJyUOep8WztybQj0+6Z/Gi1WjZdKlr+PqzpMGOGWu3p6oASKl8H9E8eLT2Yemgq3u28yU7OZkWfFURuiNTrGlqNlmPfHuOrwK84v+Y8CpWCri935blzzxl1derZm2fJKsjC2dbZ4qedzYmdix19P+zLqP89YNNrLaTHphMTHlM1gaFHEXTt2rVRKBQoFAqaNm1a4g1ZrVaTmZnJs88+a5QgLVl4THipouWyzOs7j9ldZht0autB8jLyOPjJQQBC3gwxSRGauLdaNrX4ffzvPPnbk/x05icmbJhAcnYys7rMMnVoetGoNWybs+2+5+TczsHWxfae3z+ddFr3IaFvg76GDrFGKe4IbYhC6LI4eTsxec9k1o5fy+XNl1k9ejWDPh1El9kPnsa9efYmm57ZROyBotFwn44+DP9+OF5tjb/ir7j/T1ffrigV8rvQ0NQF5ZsdykjIMHIkfyt3ArRw4UK0Wi1Tpkzhvffew8XFRfc9GxsbAgIC6NpVlg3+U3k3uAxwDajS5Afg2DfHyLmdg1sTN5n6MlPWKmv++/B/qetQl0WHFzF762xuZt3kw74fWkwNTEx4TImCx7KkxxV98gvoHVDm94tXfw1oNKDKf06qm3be7YCiBOheI/qVZeNow/hfx/PHC39w/NvjbH1xK6nXUhn4yUCAUiuA1Plq9n6wlwP/PoCmUIONow195/Wl43Mdq+yD2d0JkDC88tZsGaO2617KnQBNmjQJgAYNGtCtWzesrSvXb6GmKO8KrareCDM/K58D/yn6gQ/5V0iJJYrCvCgVSj4b9BmetTx54883mLdvHjezbvLNsG+wUpr/rszl/UR3v/N0019NZPqrslq6t8RKacXtnNvEpMVQ3/XBu6NXhNJKydCvh1K7QW12vLaDw4sOE3cwjvT4dDLi//5/7VDXAYVKQVZSFgDNRzZn8BeDcfY13Arc8pAO0MblH+KPs68z6fHplFERoqsB8g+putYfev/27NWrF2q1mnXr1hEZWTS327JlSx566CFUKsMsqaxOijfCjE+PL7MOSIECX2ffKt8I8/h3x8m+lU3thrVp/XjrKr230J9CoWBuyFzca7nzzKZn+CHiB5Jzklk1epXZ74dV2U9+SZlJHIkvatIp9T+VZ2tlS0v3lpxKOkVEYoTREiAo+nfb/dXuuNR3YcOEDcQfiS91TnZyUYsPOzc7RiwZQfORzY0Wz70kZCQQnRqNAgWdfTtX+f1rAqVKSeiiUFaPWV206uvut8O/BiFDF4ZWaSmG3ne6cuUKgYGBTJw4kfXr17N+/XomTJhAy5YtiYqy7J1hjeF+G2EWf70wdKHB+nGUR0FOAfs/3g/I6I+leardU6wbtw5blS2/XviV0B9DSctNM3VY91X8ye8e+8AWffLzu/cnv82XN6NFSwefDlU+Ulpd6eqAjFAIXZYWY1pgV/v+ibq1vTVNhzetknj+6WBcUS1kkGcQzrZVO/JUkwSOCmTc2nE41yv5d+zs62yQJfD60vudb+bMmTRq1IjY2FhOnDjBiRMniImJoUGDBsycOdMYMVq8e22E6evsy9pxaxkV+IDqeAM7sfgEWUlZuNR3ofUTMvpjaUY2H8nWCVtxtnVmz/U99Frei8TMRFOHdU/Fn/yA0klQOT75yfSX4RmyI3R5xITHkH0r+77nZMRnVOkKoLvppr98ZfrL2AJHBTLr2iwm7ZrEqJWjmLRrErOiZ1V58gMVmALbs2cPhw4dKrHkvU6dOnz00Ud0797doMFVJ/faCLMqR34ACnML2b/gr9GfN0JQWcu0pSXqFdCLPZP3EPpjKKeSTtF9aXe2TdhGI7dGpg6tTMWf/MrsALvw3h1gcwtz2RZVtIJseDPZ/sJQqjoBMkQdmDFJ/U/VUqqU91zwUJX0ToBsbW3JyCj9jzQzMxMbG/Pd9MwclLURZlWLWBpBxo0MnP2caTOpjUljEZXT1qst+6fsZ+CPA7l65yrdl3YnbEKY2W4RETgqkGYjmum1B9Dua7vJKsjCx8lH96YtKq+NV9HPflx6HMnZyUbfeNccVwAVyy3M5XjCcUASoJpG7ymwYcOG8fTTT3P48GG0Wi1arZZDhw7x7LPP8tBDDxkjRmEghXmF7Ju/D4Aer/fAytb8VxCJ+2vk1oj9U/bTxrMNSVlJ9Freiz3X9pg6rHsq/uQX9GgQAb0DHljwePf0l6Us+7cEzrbONHZrDFRNHVBl68CM6UTCCfLV+XjU8qBh7YZVfn9hOnonQJ9//jmNGjWia9eu2NnZYWdnR/fu3WncuDGLFi0yRozCQE6tOEV6XDpOPk4ET5FP09WFl6MXeybvoWf9nqTnpTPox0H8euFXU4dVaVqt9u/d32X6y+CqchqssnVgxnT39Jck2TWL3v/aXF1d+e2337h48SJr165l7dq1XLx4kQ0bNpRojijMi7pATfi8cAC6v9YdKzsZ/alOXOxc2DphKyObjyRPncfo1aNZcmKJqcOqlLM3zxKTFoOdlZ10fzaCqq4DMrcVQMWkALrm0vtdcN++ffTo0YMmTZrQpInx9mURhnX6f6dJu55GLc9atJvWztThCCOws7Jjzdg1PLvpWZZELOGpjU9xK/sWr3V/zSI/2RaP/vRv2B8HawcTR1P9VPVSeKhYHZgxabVa3RJ4qf+pefROgPr27Uu9evV49NFHmTBhAi1ayKZx5k5TqCH8//4a/Xm1O9b20sW7urJSWrF4+GI8ankwf9985u6cS1JmEp8M+sTi9jfSTX81lekvYygeAbqUconM/EwcbRyr5L7msgII4FrqNRIzE7FWWtPep72pwxFVTO/fiDdu3OCll15iz549tGrVirZt2/Lvf/+buLgHb/gpTOPMyjPcuXoHB3cH2j8jP+TVnUKhYF6/eXw26DMAFh5eyMQNEylQF5g4svK7mXWTw3GHARjaZKiJo6mePB098XHyQYuW00mnTR2OSRRPf7Xzbmf2HdWF4emdANWtW5fnn3+e/fv3ExUVxdixY1mxYgUBAQH07Svz9OZGU6hh74d7Aej2cjdsakmrgppidpfZ/Pjwj1gprfjpzE+M+HkEWflZpg6rXP64/AdatLTzbleqgagwHF0dUBVOg5kT6f9Ts1VqTLxBgwa8/vrrfPTRRwQFBbFnj/7Lb+Pj45kwYQJ16tTB3t6eoKAgjh07pvt+UlISkydPxsfHBwcHB0JDQ7l8+fJ9r7l8+XIUCkWJh51dzczuz/5yltuXb2Nfx56Oz3U0dTiiij3e+nF+H/879lb2bLmyhf7/68/tnNumDuuBiqe/pPuzcRUnQCcSTpg4EtM4ECcJUE1W4QRo//79PPfcc3h7e/PYY4/RqlUrNm/erNc17ty5Q/fu3bG2tmbLli2cP3+eTz75hNq1awNFBWojR47k6tWr/Pbbb0RERFC/fn369+9PVtb9P8k6OzuTkJCge1y/fr2iL9ViadQawj8sqv3pOqcrNo4y+lMTDW4ymJ0Td1LbrjaH4g4RsiyEuHTznbLOK8yT7s9VRFcIXUUrwcxJRl6GbupPEqCaSe8i6Llz5/Lzzz9z48YNBgwYwKJFixgxYgQODvqv0liwYAF+fn4sW7ZMd6xBgwa6P1++fJlDhw5x9uxZWrZsCcA333yDl5cXq1at4qmnnrrntRUKBV5eXnrHVJ2cX3ue5AvJ2Lna0en5TqYOR5hQV7+u7Juyj0E/DuL8rfN0W9KNbU9so3ndqt95+0H2XN9DZn4m3o7etPOWFYvGVDwCdPbmWfLV+dioas6HpCPxR9BoNdR3qY+Pk4+pwxEmoPcI0N69e3nllVeIj49n06ZNPProoxVKfgB+//13OnTowNixY/Hw8CA4OJjFixfrvp+XlwdQYvpKqVRia2vLvn377nvtzMxM6tevj5+fHyNGjODcuXP3PDcvL4/09PQSD0un1WjZ+0FR7U+XF7tg62xr4oiEqbVwb8H+KftpVqcZsemx9FjagyPxR0wdVinF3Z+HNhlqcSvXLE2AawCudq4UaAo4f+u8qcOpUlL/I/T+7VI89VW3buX3jrl69SrffPMNTZo0YevWrUyfPp2ZM2eyYsUKAJo3b46/vz9z587lzp075Ofns2DBAuLi4khISLjndZs1a8bSpUv57bff+PHHH9FoNHTr1u2eK9Xmz5+Pi4uL7uHn51fp12ZqkRsiuXXuFrYutnSe2dnU4Qgz4e/iz74p++jo05GUnBT6ruirm24yB9L9uWopFArd3nE1rRBa6n9EhT5eXbx4keeff55+/frRr18/nn/+eS5evKj3dTQaDe3atWPevHkEBwfz9NNPM23aNL799lsArK2tWb9+PZcuXcLNzQ0HBwd27drF4MGDUSrvHXrXrl2ZOHEibdu2pVevXqxfvx53d3e+++67Ms+fO3cuaWlpukdsbKzer8WcaDVa9r5fNPrTeVZn7FxrZgG4KFtdh7r8OelPBjQcQFZBFsNWDmPVmVWmDguAc7fOcS31GnZWdvRv2N/U4dQIVd0R2hxotBoOxkoDxJpO7wRo3bp1tGrViuPHj9OmTRvatGnDiRMnaNWqFevWrdPrWt7e3qUaKQYGBhITE6P7un379pw8eZLU1FQSEhIICwsjJSWFhg3Lv2mdtbU1wcHBXLlypczv29ra4uzsXOJhyS5uvEjS6SRsnGzoMquLqcMRZsjRxpFNj21ifKvxFGgKeHz943xx+AtTh6Wb/urboK90f64iNTEBirwVSVpeGg7WDrT2bG3qcISJ6F0E/eqrrzJ37lzef//9EsffeecdXn31VUaPHl3ua3Xv3r3UyNGlS5eoX79+qXOL9xm7fPkyx44d44MPPij3fdRqNWfOnGHIkCHlfo6l0mr/Hv3p9EIn7N3sTRyRMFc2Kht+GvUTde3r8uXRL5kZNpNb2bd4r/d7Jts6Q7o/V73ilWAnE0+i0WpqRN1Vcf1P53qdsVLKvog1ld7/0hMSEpg4cWKp4xMmTLhvXU5ZXnzxRQ4dOsS8efO4cuUKK1eu5Pvvv2fGjBm6c9asWcPu3bt1S+EHDBjAyJEjGThwoO6ciRMnMnfuXN3X77//Ptu2bePq1aucOHGCCRMmcP369fuuGqsuLv9xmYQTCVjXsqbri11NHY4wc0qFks8Hf84HfYo+UHyw9wOmb56OWqOu8lhuZd3STUtI9+eq07xuc+ys7MjMzyTqdpSpw6kSsv+XgAokQL179yY8PLzU8X379hESEqLXtTp27MiGDRtYtWoVrVq14oMPPmDhwoU8/vjjunMSEhJ44oknaN68OTNnzuSJJ55g1aqS9QoxMTElkq87d+4wbdo0AgMDGTJkCOnp6Rw4cKDa71t29+hPxxkdcagrUwjiwRQKBW/2fJNvh36LAgXfHf+OR9Y+Qm5hbpXGseXKFrRoaevVFj8Xy1+IYCmslFYEeQQBNWcarHgEqKuvfEisyRRarVarzxO+/fZb3n77bcaNG0eXLkX1JYcOHWLNmjW89957+Pj83U/hoYceMmy0VSQ9PR0XFxfS0tIsqh7oytYr/BT6E1b2Vsy+NptaHrVMHZKwMOvOr+Ox9Y+Rr86nT0Affh3/K862VfMzMHbNWNaeX8tbPd/i/T7vP/gJwmCe3fQs3x3/jte7v878/vNNHY5RJWcn4/5v96I/v5JMHYc6Jo5IGJI+7996T34+99xzAHz99dd8/fXXZX4Pij5VqtVVP4xeU2m1Wva8V7QVSYfpHST5ERUyusVowuzDGPHzCHZd20Xv5b3Z8vgWPB09jXrffHU+W69sBWBYU9n+oqrVpELoQ3GHgKKpP0l+aja9p8A0Gk25HpL8VK3oP6OJOxiHlZ0V3V6WeW1RcX0a9GH35N24O7gTkRhBj2U9iL4TbdR77r2+l4z8DLwcvejg08Go9xKlFRdCn0g4gZ6TAhZH1wDRV35P1nTVv9y/hiiu/Wn3dDucvJ1MHI2wdO2827F/yn4CXAO4cvsK3ZZ20+2bZAwbLxat/pLuz6YR5BGESqHiVvYtbmTcMHU4RiUdoEUxvafA/rn8/Z/efvvtCgcjKubanmtc33sdlY2K7q92N3U4oppoUqcJ+6fsJ/THUM7cPEPPZT3Z+OhGQurrt9jhQe7u/izTX6Zhb21P87rNOXfrHBGJEdRzrmfqkIyiQF2g2/5FEiChdwK0YcOGEl8XFBQQHR2NlZUVjRo1kgTIBIprf4KfCsa5nuUUbQvz5+Pkw94n9/LQqocIjwln4I8D+WXMLzzUzHALHCKTI4lOjcZWZSvdn00o2Du4KAFKiKi2ieippFPkFOZQ2642zeo2M3U4wsT0HmuOiIgo8Th79iwJCQn069ePF1980Rgxivu4Hn6da7uuobRW0uO1HqYOR1RDrnaubJ2wleFNh5NbmMuoX0axLGKZwa5fPP3Vp0EfHG0cDXZdoZ+aUAitW/7u11WmWoVhaoCcnZ157733eOuttwxxOaGH4h3f2z7ZFhd/FxNHI6ore2t71j+ynsltJ6PWqpny+xQ+3v+xQa696XLR9hfS/dm0alICJAXQAgxYBF28kaioOrEHY7m6/SpKKyUhcw1blyHEP1kprVj60FJe7fYqAK/teI2Xt72MRqup8DVTslN0b0rVddrFUhTvCn8t9Rp3cu6YNhgjkQJocTe9a4A+//zzEl9rtVoSEhL43//+x+DBgw0WmHiw4tGf1hNb4xrgatpgRI2gUChYMGABHrU8eHn7y3xy8BNuZd/ih+E/YK2y1vt6W65sQaPV0NqzNf4u/kaIWJRXbfvaBLgGcC31GicTT9KnQR9Th2RQcelxxKbHolKo6Fivo6nDEWZA7wTos88+K/G1UqnE3d2dSZMmldiPSxhX/NF4rmy5gkKlIOQNGf0RVeulbi/hXsudKb9N4b+n/ktKdgqrx67Wewd32fzUvAR7BXMt9RoRiRHVLgEq3meutWdrqTUTQAUSoOho4zZEE+WjG/2Z0Bq3Rm4mjkbURBPbTKSOfR3GrhnL5subGfC/AWx6dBO17WuX6/n56nzCroQBkgCZi2CvYDZc2FAt64Bk+kv8k5TBW6CEiAQubbyEQimjP8K0hjYdyvYntuNq58qB2AOELAshPj2+XM/dF7OP9Lx0PGp5yJSEmSjuCB2RUA0ToDhJgERJkgBZoOLRn1aPtqJOU9nLRphWd//uhD8Zjo+TD+dunaPb0m5cTL74wOdJ92fz0867HQAXki+QU5Bj4mgMJ6cghxMJJwBJgMTf5LeOhUk6ncSFDRdAASH/ktEfYR5aebTiwJQDNK3TlJi0GHos68GxG8fueb50fzZP3o7eeNTyQK1Vc+bmGVOHYzDHbhyjUFOIt6M39V3qmzocYSYkAbIwez8sGv1pOa4l7oHuJo5GiL/Vd63Pvif30cGnA8nZyfRZ0YcdV3eUee7FlItE3YnCRmXDwEYDqzhScS8KhULXD6h4xKQ6uLv+R6FQmDgaYS7KlQC1a9eOO3eK+kK8//77ZGdnGzUoUbab525yfu15AHq+2dPE0QhRmnstd/6c+Cf9GvQjMz+TIT8NYfW51aXO03V/DpDuz+ZG1xCxGtUBSf2PKEu5EqDIyEiysrIAeO+998jMzDRqUKJs4f8XDloIHB2IRysPU4cjRJmcbJ3Y/NhmxrYYS4GmgPFrx/P10a8BUGvU7L62m6UnlwIwpMkQU4YqyqArhK4mK8G0Wq2sABNlKtcy+LZt2/Lkk0/So0cPtFot//nPf3B0LPtTm2yGahzJF5I5+/NZQEZ/hPmztbJl1ehVuDu48/Wxr5nxxwz2XNvDgbgDxKXH6c77aN9H+Dr7MipwlAmjFXcrHgE6c/MMhZpCrJR6d0sxK1duXyE5Oxlbla3utQkB5UyAli9fzjvvvMOmTZtQKBRs2bIFK6vST1UoFJIAGUn4vKLRn2YjmuHV1svU4QjxQCqlii+HfIlHLQ/e3fMuq8+XngpLzExkzOoxrB23VpIgM9HIrRFONk5k5GdwIfkCrTxamTqkSike/eng0wFbK1sTRyPMSbkSoGbNmvHzzz8DRZ2fd+7ciYeHTMFUldtXbnPmp6IVGT3fktEfYTkUCgVv9nyThYcXkpqbWur7WrQoUDA7bDYjmo1ApVRVfZCiBKVCSRuvNuyL2UdEQkS1SYBk+kv8k96rwDQajSQ/VSx8XjhajZYmQ5vg097H1OEIoZfwmPAyk59iWrTEpscSHhNedUGJ+6pOO8MfjCvaAqOrb1cTRyLMTYUmd6Oioli4cCGRkZEAtGjRglmzZtGoUSODBifgztU7nPrvKUBGf4RlSshIMOh5wviqSwKUlpvG2ZtFtZNd/SQBEiXpPQK0detWWrRowZEjR2jdujWtW7fm8OHDtGzZku3btxsjxhotfH44WrWWRoMa4dvZ19ThCKE3bydvg54njK94JdjJxJNotVoTR1Nxh+MPo0VLw9oN8XKU2klRkt4jQK+//jovvvgiH330Uanjr732GgMGDDBYcDVd6vVUTi0vGv3p9XYvE0cjRMWE+Ifg6+xLfHo8Wkq/mSpQ4OvsS4i/dDY3Fy3cW2CttCY1N5VrqddoULuBqUOqEKn/Efej9whQZGQkU6dOLXV8ypQpnD9/3iBBiSL7PtqHplBDg34N8OvmZ+pwhKgQlVLFotBFQFGyc7firxeGLpQCaDNio7IhyDMIsOxpMF0C5CsJkChN7wTI3d2dkydPljp+8uRJKY42oLTYNCKWFP3ikdEfYelGBY5i7bi11HOuV+K4r7OvLIE3U5beEVqtUXMo7hAgI0CibHpPgU2bNo2nn36aq1ev0q1b0T+q/fv3s2DBAubMmWPwAGuq/R/vR1OgIaB3APV7yuZ9wvKNChzFiGYjCI8JJyEjAW8nb0L8Q2Tkx0xZeiH0uVvnyMjPwNHG0eKX8gvj0DsBeuutt3BycuKTTz5h7ty5APj4+PDuu+8yc+ZMgwdYE2XcyODE4qKNCHu+LSu/RPWhUqroHdDb1GGIciguhLbUTVGLp7+6+HaRJFuUSe8ESKFQ8OKLL/Liiy+SkZEBgJOTk8EDq8n2/3s/6jw1/j38CegdYOpwhBA1UGvP1ihQkJCZQFJmEp6OnqYOSS9S/yMeRO8aoLs5OTlJ8mNgmYmZHP/2OFA0+qNQKB7wDCGEMDxHG0ea1mkKWOY0mKwAEw9SqQRIGN6BTw5QmFuIbxdfGvZvaOpwhBA1mG5neAsrhE7KTCLqThQKFHT27WzqcISZkgTIjGTdyuLY18cAGf0RQpiepRZCF29/0dKjJa52rqYNRpgtSYDMyMFPD1KQXYBPBx8ahzY2dThCiBrOYhOgWNn/SzyYXglQQUEB/fr14/LlywYLID4+ngkTJlCnTh3s7e0JCgri2LFjuu8nJSUxefJkfHx8cHBwIDQ0tFz3X7NmDc2bN8fOzo6goCD++OMPg8VsDNkp2Rz98iggoz9CCPNQPAV25fYV0vPSTRxN+R2Ik/of8WB6JUDW1tacPn3aYDe/c+cO3bt3x9rami1btnD+/Hk++eQTateuDYBWq2XkyJFcvXqV3377jYiICOrXr0///v3Jysq653UPHDjAo48+ytSpU4mIiGDkyJGMHDmSs2fPGix2Qzu08BD5mfl4tfWi6bCmpg5HCCGo61AXX+eiPQhPJZ4ycTTlk6/O52h80YdJSYDE/eg9BTZhwgSWLFlikJsvWLAAPz8/li1bRqdOnWjQoAEDBw7U7Sp/+fJlDh06xDfffEPHjh1p1qwZ33zzDTk5Oaxateqe1120aBGhoaG88sorBAYG8sEHH9CuXTu+/PLLMs/Py8sjPT29xKMq5dzJ4cjnRwAZ/RFCmBdLmwaLSIggT51HHfs6NHFrYupwhBnTOwEqLCzkm2++oUOHDjzzzDPMmTOnxEMfv//+Ox06dGDs2LF4eHgQHBzM4sWLdd/Py8sDwM7O7u+AlUpsbW3Zt2/fPa978OBB+vfvX+LYoEGDOHjwYJnnz58/HxcXF93Dz69q9906/Plh8tLz8AjyoPmI5lV6byGEuB9LS4DuXv4uHybF/eidAJ09e5Z27drh5OTEpUuXiIiI0D3K2iPsfq5evco333xDkyZN2Lp1K9OnT2fmzJmsWLECgObNm+Pv78/cuXO5c+cO+fn5LFiwgLi4OBISEu553cTERDw9Szbt8vT0JDExsczz586dS1pamu4RGxur1+uojNy0XA4vPAxAz7d6olDKD6wQwny0824HWM5SeKn/EeWldyfoXbt2GezmGo2GDh06MG/ePACCg4M5e/Ys3377LZMmTcLa2pr169czdepU3NzcUKlU9O/fn8GDB6PVag0Wh62tLba2tga7nj6OfHGE3NRc3Fu402J0C5PEIIQQ91JcCH3u1jnyCvOwtTLN78ry0Gq10gBRlFuFl8FfuXKFrVu3kpOTA1ChhMTb25sWLUq+6QcGBhITE6P7un379pw8eZLU1FQSEhIICwsjJSWFhg3v3STQy8uLpKSkEseSkpLw8vLSO0ZjysvI4+CnRdNyIW+GyOiPEMLs+Dn74WbvRqGmkHO3zpk6nPuKSYvhRsYNrJRWdPDpYOpwhJnTOwFKSUmhX79+NG3alCFDhuimoqZOncpLL72k17W6d+/OxYsXSxy7dOkS9euX3v3cxcUFd3d3Ll++zLFjxxgxYsQ9r9u1a1d27txZ4tj27dvp2tW8ekIc/eoouXdyqdO0Di3HtTR1OEIIUYpCodDVAZn7xqjFoz/BXsE4WDuYOBph7vROgF588UWsra2JiYnBweHvf2CPPPIIYWFhel/r0KFDzJs3jytXrrBy5Uq+//57ZsyYoTtnzZo17N69W7cUfsCAAYwcOZKBAwfqzpk4caJuZ3qAWbNmERYWxieffMKFCxd49913OXbsGM8//7y+L9do8jPzOfjJ36M/SpX0pBRCmCddIbSZ1wHJ9JfQh941QNu2bWPr1q34+vqWON6kSROuX7+u17U6duzIhg0bmDt3Lu+//z4NGjRg4cKFPP7447pzEhISmDNnDklJSXh7ezNx4kTeeuutEteJiYlBqfw7gejWrRsrV67kzTff5I033qBJkyb8+uuvtGrVSt+XazTHvj1GdnI2bo3dCHo0yNThCCHEPen2BDPzlWBSAC30oXcClJWVVWLkp9jt27crVEg8bNgwhg0bds/vz5w5k5kzZ973Grt37y51bOzYsYwdO1bveKpCQXYBB/5d9IMa8q8QlFYy+iOEMF/FI0Cnkk6h1qhRKVUmjqi0rPwsXbNG2QJDlIfe77whISH897//1X2tUCjQaDR8/PHH9OnTx6DBVTcatYZru6+x6dlNZN3MwiXAhaDHZfRHCGHemtZpioO1A9kF2Vy+bbitkAzp6I2jqLVqfJ198XOp2l5uwjLpPQL08ccf069fP44dO0Z+fj6vvvoq586d4/bt2+zfv98YMVYLkesjCZsVRnrc312m89LyuLTxEoGjAk0YmRBC3J9KqaK1Z2sOxR0iIiGC5nXNr2Gr1P8Ifek9AtSqVSsuXbpEjx49GDFiBFlZWYwaNYqIiAjdFhaipMj1kawes7pE8gOQm5rL6jGriVwfaaLIhBCifMy9I7QuAfKVBEiUj94jQFC0JP1f//qXoWOpljRqDWGzwqCsNklaQAFhs8NoNqKZrAQTQpgtc06ANFoNB+OKVtXKCJAorwolQHfu3GHJkiVERhaNXLRo0YInn3wSNzc3gwZXHcSEx5Qa+SlBC+mx6cSExxDQO6DK4hJCCH3oVoIlRKDVas1qn61LKZe4nXMbeyt72nq1NXU4wkLoPeSwd+9eAgIC+Pzzz7lz5w537tzh888/p0GDBuzdu9cYMVq0jIQMg54nhBCm0MqjFVZKK1JyUohLjzN1OCUUT391rNcRa5W1iaMRlkLvBGjGjBk88sgjREdHs379etavX8/Vq1cZP358iQaGooiTt5NBzxNCCFOws7KjhXvR1kXmNg0m9T+iIvROgK5cucJLL72ESvV3HwiVSsWcOXO4cuWKQYOrDvxD/HH2dYZ7jRYrwNnPGf8Q/yqNSwgh9GWuHaFlBZioCL0ToHbt2ulqf+4WGRlJmzZtDBJUdaJUKQldFFr0xT+ToL++Dl0YKgXQQgizZ46F0LdzbhOZXPSe1NVPGiCK8itXEfTp06d1f545cyazZs3iypUrdOnSBYBDhw7x1Vdf8dFHHxknSgsXOCqQcWvHleoD5OzrTOjCUOkDJISwCMWF0Oa0KeqhuENAUbPGug51TRyNsCQKrVZb1gLtEpRKJQqFggedqlAoUKvVBgvOVNLT03FxcSEtLQ1nZ2eDXVej1hATHkNGQgZO3k74h/jLyI8QwmKk56Xj8pELAMmvJFPHoY6JI4I3/3yT/wv/Pya1mcTykctNHY4wMX3ev8s1AhQdHW2QwGo6pUopS92FEBbL2daZRrUbEXUniojECPo37G/qkKT+R1RYuRKg+vXrGzsOIYQQFiDYO7goAUowfQJUqCnkSPwRQBIgob8KNUK8ceMG+/bt4+bNm2g0mhLfe9DO7UIIISxXsFcwa8+vNYtC6DNJZ8gqyMLZ1lm3RF+I8tI7AVq+fDnPPPMMNjY21KlTp0Q3UIVCIQmQEEJUY+a0Eqx4+qurb1eUCqmnFPrROwF66623ePvtt5k7dy5KpfyDE0KImqR4JdjF5Itk5WdRy6aWyWI5ECf1P6Li9M5gsrOzGT9+vCQ/QghRA3k5euHl6IUWLaeTTj/4CUYkBdCiMvTOYqZOncqaNWuMEYsQQggL0M67HWDaabAbGTe4lnoNpUJJp3qdTBaHsFx6T4HNnz+fYcOGERYWRlBQENbWJTee+/TTTw0WnBBCCPMT7BXMH5f/MOmWGAdjDwIQ5BGEs63h+rWJmqNCCdDWrVtp1qwZQKkiaCGEENWbORRCy/SXqCy9E6BPPvmEpUuXMnnyZCOEI4QQwtwVF0KfuXmGAnUB1irrBzzD8KQAWlSW3jVAtra2dO/e3RixCCGEsAANXBvgYutCvjqf87fOV/n9cwtzOX7jOFC0BF6IitA7AZo1axZffPGFMWIRQghhARQKBW292gKmmQY7fuM4BZoCPGp50LB2wyq/v6ge9J4CO3LkCH/++SebNm2iZcuWpYqg169fb7DghBBCmKdgr2D2XN9DREIEk9tOrtJ7H4wrKoDu5tdNak9FhemdALm6ujJq1ChjxCKEEMJCFNcBmWIESFcA7Sv1P6Li9E6Ali1bZow4hBBCWJDilWAnE0+i0WqqbCsKrVYrK8CEQUg7ZyGEEHprXrc5tipbMvIzuHrnapXdNzo1mqSsJKyV1rT3aV9l9xXVj94jQA0aNLjvnOvVq1X3gyCEEMI0rFXWBHkGcezGMSISImjs1rhK7ls8+tPepz12VnZVck9RPemdAM2ePbvE1wUFBURERBAWFsYrr7xiqLiEEEKYuWCv4KIEKDGCsS3HVsk9pf5HGIreCdCsWbPKPP7VV19x7NixSgckhBDCMpiiI7TU/whDMVgN0ODBg1m3bp2hLieEEMLM6TZFraI9wdLz0jlz8wwAXf2kAaKoHIMlQGvXrsXNzc1QlxNCCGHmgjyDUCqUJGUlkZCRYPT7HYk/gkarIcA1AB8nH6PfT1RveidAwcHBtGvXTvcIDg7G29ubN954gzfeeEPvAOLj45kwYQJ16tTB3t6eoKCgElNpmZmZPP/88/j6+mJvb0+LFi349ttv73vN5cuXo1AoSjzs7KRYTgghDMnB2oHmdZsDVTMNJtNfwpD0rgEaOXJkia+VSiXu7u707t2b5s2b63WtO3fu0L17d/r06cOWLVtwd3fn8uXL1K5dW3fOnDlz+PPPP/nxxx8JCAhg27ZtPPfcc/j4+PDQQw/d89rOzs5cvHhR97V0CxVCCMML9grm/K3zRCREMKTJEKPeqzgBkv2/hCHonQC98847Brv5ggUL8PPzK9FcsUGDBiXOOXDgAJMmTaJ3794APP3003z33XccOXLkvgmQQqHAy8vLYLEKIYQoLdgrmJ/O/MSJxBNGvY9Gq+FQ3CFARoCEYZi0EeLvv/9Ohw4dGDt2LB4eHgQHB7N48eIS53Tr1o3ff/+d+Ph4tFotu3bt4tKlSwwcOPC+187MzKR+/fr4+fkxYsQIzp07d89z8/LySE9PL/EQQgjxYLotMYxcCB15K5K0vDQcrB1o7dnaqPcSNUO5EyClUolKpbrvw8pKvwGlq1ev8s0339CkSRO2bt3K9OnTmTlzJitWrNCd88UXX9CiRQt8fX2xsbEhNDSUr776ip49e97zus2aNWPp0qX89ttv/Pjjj2g0Grp160ZcXFyZ58+fPx8XFxfdw8/PT6/XIYQQNVXxrvDRqdGk5qYa7T7F01+d63XGSqn35IUQpZT7X9GGDRvu+b2DBw/y+eefo9Fo9Lq5RqOhQ4cOzJs3DygqsD579izffvstkyZNAooSoEOHDvH7779Tv3599u7dy4wZM/Dx8aF///5lXrdr16507fr3HHG3bt0IDAzku+++44MPPih1/ty5c5kzZ47u6/T0dEmChBCiHNzs3ajvUp/radc5mXiS3gG9jXKfA3FSAC0Mq9wJ0IgRI0odu3jxIq+//jobN27k8ccf5/3339fr5t7e3rRo0aLEscDAQF0/oZycHN544w02bNjA0KFDAWjdujUnT57kP//5zz0ToH+ytrYmODiYK1eulPl9W1tbbG1t9YpdCCFEkWDvYK6nXSciIcJ4CZCsABMGVqEaoBs3bjBt2jSCgoIoLCzk5MmTrFixgvr16+t1ne7du5dYqQVw6dIl3XUKCgooKChAqSwZpkql0mu0Sa1Wc+bMGby9vfWKTwghxIMZuyN0cnYyl1IuAdDFt4tR7iFqHr0mUtPS0pg3bx5ffPEFbdu2ZefOnYSEhFT45i+++CLdunVj3rx5jBs3jiNHjvD999/z/fffA0VL2Xv16sUrr7yCvb099evXZ8+ePfz3v//l008/1V1n4sSJ1KtXj/nz5wPw/vvv06VLFxo3bkxqair//ve/uX79Ok899VSFYxVCCFE2YydAB2MPAhBYNxA3e2m4Kwyj3AnQxx9/zIIFC/Dy8mLVqlVlTonpq2PHjmzYsIG5c+fy/vvv06BBAxYuXMjjjz+uO+fnn39m7ty5PP7449y+fZv69evzf//3fzz77LO6c2JiYkqMEt25c4dp06aRmJhI7dq1ad++PQcOHCg13SaEEKLyileCRd6KJKcgB3tre4NeX6a/hDEotFqttjwnKpVK7O3t6d+/PyqV6p7nrV+/3mDBmUp6ejouLi6kpaXh7Oxs6nCEEMKsabVaPP/jya3sWxx56ggd63U06PV7Le/F3ut7WfLQEqYETzHotUX1os/7d7lHgCZOnCjdlIUQQpSiUCgI9g5mW9Q2IhIjDJoAFagLOBJ/BJARIGFY5U6Ali9fbsQwhBBCWLJgr78SIAM3RDyZeJLcwlxq29WmaZ2mBr22qNlM2glaCCFE9WCsQmjd/l9+XVEq5C1LGI78axJCCFFpxYXQp5NOU6gpNNh1dQ0QfWX6SxiWJEBCCCEqrbFbYxxtHMkpzOFi8sUHP6GcipfAS/2PMDRJgIQQQlSaUqGkjWcbwHDTYLFpscSmx6JSqAy+skwISYCEEEIYhK4OyECF0AfjikZ/2ni1wdHG0SDXFKKYJEBCCCEMorgOyFAjQLoGiFL/I4xAEiAhhBAGcfdKsHL22L0v6QAtjEkSICGEEAbR0qMl1kprUnNTuZ52vVLXyi7I1o0kSQIkjEESICGEEAZho7KhpUdLoPJ1QMduHKNQU4iPkw/+Lv6GCE+IEiQBEkIIYTDtvNoBla8Dunv6S7ZhEsYgCZAQQgiDMVQhtK4DtG/XSsckRFkkARJCCGEwhlgKr9VqpQBaGJ0kQEIIIQymjVcbFCiIz4jnVtatCl3j8u3LpOSkYKuy1SVUQhiaJEBCCCEMxtHGkSZ1mgAVnwYrHv3p4NMBWytbg8UmxN0kARJCCGFQlZ0Gk/2/RFWQBEgIIYRBFSdAJxJPVOj5uh3gJQESRiQJkBBCCIPSrQSrwAhQam4q526eA2QFmDAuSYCEEEIYVPEI0OXbl8nIy9DruYfjDqNFS6PajfB09DRGeEIAkgAJIYQwMPda7tRzqgfAqaRTej1Xlr+LqiIJ0P+3d99RUZxtG8DvBZGiFDUC0kQUREQEKSoWRLFjYgnBBAOJQUQlKq+aUI6CsRuxxxKNYnlfTUCJWIK9xhIboLEhKgICKhZAQGD3+v7g2wkb0BgF1t29f+d4jjszLPfMsjPXPPPMPIwxxmrd214G4/4/rL5wAGKMMVbrqo4M/6bEEjGdzTpLRByAWN3jAMQYY6zWvU0AuvrwKhWVFZFuQ11q37x9XZXGGBFxAGKMMVYHOrWoHBT1z4d/Upm47I1+Rtr/p7NZZ1JXU6+z2hgj4gDEGGOsDljoW1ATrSZULikXbmv/J0L/HzO+/MXqHgcgxhhjtU4kEv3rkeH5DjBWnzgAMcYYqxP/ZkiMvKI8uvP0DolIRJ3NOtd1aYxxAGKMMVY3/k1H6DNZleN/tTdsTwZaBnVZFmNExAGIMcZYHZFeAkvOTSaxRPzaZYXLX9z/h9UTDkCMMcbqRNtmbUm7gTa9KH9Bt5/cfu2y3P+H1TcOQIwxxuqEupo6ORg5ENHrL4O9rHhJFx5cICIOQKz+cABijDFWZ96kI/Tl3Mv0UvySPtD5gNo0bVNfpTEVJ/cAlJ2dTaNGjaJmzZqRtrY2dejQgS5cuCDMLyoqopCQEDIzMyNtbW2ys7OjNWvW/OP7xsXFka2tLWlpaVGHDh1o3759dbkajDHGavAmt8JXvfwlEonqpS7G5BqAnj59St26dSMNDQ367bff6Nq1axQTE0NNmjQRlvnPf/5DSUlJtHXrVrp+/TpNnjyZQkJCKDEx8ZXve/r0afr000/pq6++osuXL9PQoUNp6NChdPXq1fpYLcYYY/+v6p1gAGpchjtAM3kQ4VV/kfUgLCyMfv/9dzp58uQrl7G3tydfX1+aPn26MM3Z2ZkGDhxIs2fPrvFnfH196cWLF7Rnzx5hWpcuXcjR0bHG1qOXL1/Sy5cvhdcFBQVkbm5Oz58/Jz09vbdZNcYYY0RUWlFKjec2JjHElBmaSWZ6ZjLzAZDJYhPKLcql418cp54te8qpUqYMCgoKSF9f/42O33JtAUpMTCQXFxfy8fEhQ0NDcnJyonXr1sks4+7uTomJiZSdnU0A6OjRo3Tr1i3q16/fK9/3zJkz5OXlJTOtf//+dObMmRqXnzdvHunr6wv/zM3N333lGGOMkVYDLbJrbkdENfcDynieQblFudRArQG5mLjUd3lMhck1AN25c4dWr15N1tbWtH//fho3bhxNnDiRNm3aJCyzYsUKsrOzIzMzM2rYsCENGDCAfvjhB+rZ89VnCbm5uWRkZCQzzcjIiHJzc2tcPjw8nJ4/fy78y8zMrJ0VZIwx9tp+QNLLX07GTqSjoVOvdTHV1kCev1wikZCLiwvNnTuXiIicnJzo6tWrtGbNGgoICCCiygB09uxZSkxMpJYtW9KJEydowoQJZGJiUq2V521pamqSpqZmrbwXY4wxWU7GTrQ5ZfNrAxDf/s7qm1wDUIsWLcjOzk5mWrt27WjHjh1ERFRSUkIRERGUkJBAgwcPJiIiBwcHSk5OpkWLFr0yABkbG1NeXp7MtLy8PDI2Nq6DtWCMMfY6r7sVXjoEBgcgVt/kegmsW7dudPPmTZlpt27dopYtWxIRUXl5OZWXl5OammyZ6urqJJFIXvm+Xbt2pcOHD8tMO3jwIHXt2rWWKmeMMfamHI0diaiyv8+TkifC9KKyIkrJTSEiDkCs/sk1AIWGhtLZs2dp7ty5dPv2bfrf//5HP/74I02YMIGIiPT09MjDw4OmTZtGx44do7t371JsbCxt3ryZhg0bJryPv78/hYeHC68nTZpESUlJFBMTQzdu3KDo6Gi6cOEChYSE1Ps6MsaYqtPX0ierJlZEVDkumNT57PMkhpjM9cyr3R3GWF2TawBydXWlhIQE2rZtG9nb29OsWbNo6dKl5OfnJyyzfft2cnV1JT8/P7Kzs6P58+fTnDlzKDg4WFjm/v37lJOTI7x2d3cXwlTHjh0pPj6efv31V7K3t6/X9WOMMVZJehnsUs4lYRr3/2HyJNc+QERE3t7e5O3t/cr5xsbGtHHjxte+x7Fjx6pN8/HxIR8fn3ctjzHGWC1wMnaiHdd3yHSEPp3FAYjJj9yHwmCMMab8hFvh/78jtAQSOpPJHaCZ/Mi9BYgxxpjyk14Cu5l/k4rLiynjWQY9LX1K2g20qaNRRzlXx1QRtwAxxhircy10W5BRIyOSQEKpealC/x9XU1fSUNeQc3VMFXEAYowxVi+qXgbjAVCZvHEAYowxVi+qjgzPHaCZvHEfIMYYY/WiU4tORER05O4RSn+aTkREXc35AbVMPrgFiDHGWL2QtgBJw4+Znhk10Woiz5KYCuMAxBhjrF5czr1MIhIJr7MKsshymSXtvL5TjlUxVcUBiDHGWJ3beX0nfRL3CYEgMz27IJs+/uVjDkGs3nEAYowxVqfEEjFNSppULfwQkTBtctJkEkvE9V0aU2EcgBhjjNWpk/dPUlZB1ivng0CZBZl08v7JeqyKqToOQIwxxupUTmHOPy/0L5ZjrDZwAGKMMVanWui2qNXlGKsNHIAYY4zVqR4WPchMz0zmDrCqRCQicz1z6mHRo54rY6qMAxBjjLE6pa6mTssGLCMiqhaCpK+XDlhK6mrq9V4bU10cgBhjjNW54e2GU/wn8WSqZyoz3UzPjOI/iafh7YbLqTKmqkQAqt+XqOIKCgpIX1+fnj9/Tnp6evIuhzHGlIZYIqaT909STmEOtdBtQT0senDLD6s1/+b4zWOBMcYYqzfqaurUy7KXvMtgjC+BMcYYY0z1cABijDHGmMrhAMQYY4wxlcMBiDHGGGMqhwMQY4wxxlQOByDGGGOMqRwOQIwxxhhTORyAGGOMMaZyOAAxxhhjTOXwk6BrIB0dpKCgQM6VMMYYY+xNSY/bbzLKFwegGhQWFhIRkbm5uZwrYYwxxti/VVhYSPr6+q9dhgdDrYFEIqEHDx6Qrq4uiUSiWn3vgoICMjc3p8zMTJUcaFXV15+ItwGvv2qvPxFvA1Vff6K62wYAqLCwkExMTEhN7fW9fLgFqAZqampkZmZWp79DT09PZf/wiXj9iXgb8Pqr9voT8TZQ9fUnqptt8E8tP1LcCZoxxhhjKocDEGOMMcZUDgegeqapqUlRUVGkqakp71LkQtXXn4i3Aa+/aq8/EW8DVV9/ovdjG3AnaMYYY4ypHG4BYowxxpjK4QDEGGOMMZXDAYgxxhhjKocDEGOMMcZUDgcgxhhjjKkcDkCMMcYYUzkcgOqAKj1ZQCKR1DhdlbYBY4xJARD2f6q6H0xOTqaKigoier+3AQegWpScnExFRUW1PoDq+0oikQiDzV2/fp1u375N6enpREQqsw0Yq8n7vNNntUt6Eig94ItEIrpz547wf1VTXFxMnp6eNHXqVCJ6v7cBB6BaMn36dAoNDaU9e/aQWCxW+h0gACH8RERE0IgRI8jd3Z369u1LERERcq6ufkk/6/z8fHr48GGN85SZdB1zcnLo+vXr9Pz5cyovLyeiV7cQKhPp+mdmZtK1a9coMzOTSktL5VxV/ZJug/T0dLp9+zZlZGRUm6es1NTU6NatWzRt2jQiIoqLi6MOHTrQzZs35VxZ/aja4kVEpKOjQ8uWLaMTJ07QgQMH5FjZGwB7Z+Hh4fjggw9w4MAB5Ofny8wTi8Vyqqp+LFy4EE2bNsXBgweRlJSEtWvXolGjRhgzZoy8S6tXO3fuhKurK8zNzfH111/j/PnzwjyJRCLHyuqWdN0SEhJgb28PY2NjuLm5ISoqSvguKPN3QLr+O3fuhLW1Ndq1awcrKyuMHj0af/zxh5yrq19xcXEwNzdHixYt0K1bN6xdu1aYp8zfAQA4fPgwRCIR+vfvD3V1dcTGxgJQ/vWu6ty5c3j27BkA4MGDBxg6dCgmTZqEkpISOVf2ahyA3tH58+dhY2ODM2fOAACeP3+OW7duYe3atUhLSwOgPAeAzMxMmddlZWUYNmwYZs2aJTP9wIED0NDQwLJly+qzPLm5cOECjIyMEBUVhZiYGFhZWWHIkCE4cOCAsIwy7wj37dsHXV1dxMTEIC8vD6GhoTAxMUFQUBAePXoEQHm+AzU5duwYdHV1sXz5cgDAkiVLoKGhgQ0bNsi5sron/bvOzs5G69atsX79euzYsQOTJ0+GhYUFFi1aVG1ZZSNdr+nTp0MkEsHT0xMvXryQc1X169atWxCJRGjfvj1+/PFH5OTk4MSJE9DW1sb+/fsBvJ/7AA5A7+jKlSto2bIljhw5gitXruDrr7+GtbU1WrVqBS0tLdy8eVPeJdaK0aNHY9iwYTLTXrx4ARsbG0yePFmYVlFRAQAYP348hg4dipcvXyrtjg8A0tLSsGjRIsycOVOYlpqaCjc3N3h7e+PgwYPCdGXYDidOnJB5nZeXBy8vL8yfPx8AkJ+fDwsLC3Tu3Bn29vYYO3as0rYESddn6tSpGD16NAAgKysLVlZWCA4OFpaTnhUrq9OnTyMsLAwhISHCNsnMzMSMGTNgamqq9CGovLwcABAdHY0pU6ZAXV0dY8aMwYMHD2pcXhm3QX5+Ptzc3KCvr4/IyEj07t0bp06dwowZM2BjY4OcnBx5l1gjDkBv6fr163jx4gWysrLg7e2Ndu3aQUdHB8HBwdi+fTseP36Mdu3aKU0ryNOnT1FWVgYAePLkiTA9OjoanTp1ElrApMLDw9G7d2+l/LIDlTux/Px8mJubQ0tLS+aABwApKSlwdXXFRx99hD179sipytp1+PBh6Orq4tGjR8LnWlFRgS1btuDatWt4+PAh2rZtK2yLgIAAGBgYYOTIkUJLkKKqGt6k3wNp2B83bhxWrlyJZ8+eCS1f0u3z66+/Ij4+XlhW2RQWFiI4OBj6+vro06ePzDxpCLK0tKzWSqwMpJ+x9BKP9PWePXuEEJSbmyssX/WyuLLIyspCXl4eACA5ORl2dnZYuHAhYmNj0ahRIwwfPhwtWrTAd999h5cvX8q52uo4AL2FHTt2oHHjxvjll18AADdv3sSvv/6Ko0ePCjvHFy9ewNXVFf/73//kWWqtW79+PVq0aIFbt24BAI4fP45evXrBz88Pv//+O4DKM96+ffsiMDBQnqXWmaqh7ujRo2jTpg26du1abQeXmpoKa2tr+Pr6KkWTeElJibCzu3fvnjBdum4LFy7E4MGDhRafZcuWoV27dhgxYgSys7Prv+BaVvUS8N69e7FmzRoAlWHf3NwcZmZmmDhxotAiUFZWhlGjRuHbb78V9gvKoup34OLFiwgODoampiY2b94ss1xWVhamTJkCOzs75OfnK80JkXQ99u/fj4CAAAwcOBCBgYG4c+cOACApKQkNGjRAUFAQLl68iFmzZkFfX1/m5EHRXb9+HR07dkRAQAAuXLgAAJg/fz4mT54MiUSCP/74AwEBAVBTU4OTkxOKiorkXHF1HIDe0kcffQRLS0vEx8ejtLRUmF5SUoI7d+5g8ODBcHFxUbozv7y8PDg6OqJDhw64ffs2gMoOsH379oWRkRFcXFyE+dKdvrJ84au2egB/tQocPnwYlpaWGDVqFC5duiTzM1evXhV2isrizp07EIlE+P7772WmT5kyBW5ubsKObsqUKViwYEG1GwMUUWFhIVxcXODl5YX4+HiIRCLExcUBAAoKCtCvXz8YGBigsLAQQGX4CQ8Ph6mpqdJcBgdkWz2kQQ+o7AMyZswY2NraYuvWrTI/k52drfAtgDVJSEiAtrY2IiMjsWjRIvTo0QN6enpCq8+hQ4fQtGlTdOzYEUZGRkJIUCbLly/HqFGjoKuri59++gn//e9/0bdvX+HSf15eHg4ePCgcK943HIDewYgRI2BhYYG4uDgUFxcDAFatWoWBAwfC3d29WlO5onlVn43Hjx+jU6dOsLW1RXp6OgDgxo0b2LVrFyIiIrB69Wph51h1J6nIpDv+Q4cOYcKECfDz88OcOXOEFpH9+/fD0tISfn5+SE5Olmepda68vBzR0dFo2LChzCXeH374AW5ubhgxYgQ+//xzNGrUSGgpVHSlpaU4fPgwDA0NoampiS1btgCoDDoSiQSHDx9G+/bt0aJFC/Tp0wf9+/eHoaFhtUCsyKTfgb1792LAgAHo1q0bBg0aJNztlpaWhqCgILRt21bpWr7/Lj8/H+7u7liyZAkA4P79+zA3NxdavaXb6vr16zh+/Hi1G0gUkXSdiouLZU5qiouLsXTpUhgbGwt9YDt06KAQJz4cgN7QDz/8gKNHj1YLBcOGDYOxsTHi4+MBVF722LJlixB6FDUAVF3Ps2fPIikpCampqXj69CmAyh2ANAS9Kt0ravB7lYSEBGhpaSEwMBB9+/aFi4sLWrZsiYyMDACVd79ZW1vjww8/RGpqqpyrrT3SHd+1a9dw8uRJ3L9/HwCwaNEiiEQiLF26FEBlSIiKisLw4cMxcOBAhd4GNYX/e/fuoUmTJjAwMMBHH30kM08ikeDJkyeYM2cOpk2bhqVLlwonB8pkz5490NDQwNSpUzF79mx4eHjAxMRECIR//vknxo8fD0NDQ6GLgDK6f/8+LCwskJWVhZycHJiamiIoKEiY//PPPwv7SmUg3Qfs3r0bXl5esLW1xeDBg7Flyxbh5P/48eP4+uuv0bVrV4hEIsycOfO9b/3nAPSGHB0dYWZmht9//73aztHR0REODg7YvHmzzAeuiAFAIpHIrMM333wDExMTWFpaomHDhhg5cqRwW2N+fj6cnZ1hZ2enNGf6r/Lo0SM4Ojpi4cKFwrQrV66gX79+aNWqFR4+fAig8tp/x44dlaLPS1UJCQlo3LgxWrduDU1NTaxbtw55eXlYvHgxRCIRYmJiAPwVHN7nZ3+8qezsbKFf19atWzFlyhSkp6fjwIEDaN26NQYNGiQsq6gnOm9KIpGgqKgIffr0QVhYmMy8L7/8EiYmJkhJSQFQ2ScoNDT0vb3s8S7OnTuHJ0+eoLi4GIMGDcK6detgYWGBsWPHCn8D9+/fh5+fH/bt2yfnamvXvn370LBhQ4SFhWHNmjXo168f3NzcEBkZKYSghw8f4vTp0/D09MT169flXPE/4wD0D/773/8Kzbm9e/eGpaUlTp06JYQbiUQCX19fGBgY4OOPP5Znqe/s7820a9euRfPmzXH8+HE8ffoUe/bsQb9+/TB48GDhduhHjx7BxcUF9vb2wm2fyna7MwDcvXsXRkZGMs/2qaioQHJyMjp16oQVK1YI660MHZ6lxGIx8vPzhQfbpaWlYfbs2RCJRJg/fz5ycnKwePFiaGpqYu7cufIut1ZIJBKUlJQId/HNmzcPIpFIeLBfaWkpfv31V7Rp0wbe3t7Cz61atQrr1q2DWCx+7898X6dq7VWDnVgshpOTk3DZs2rfx65du8LHx0d4/T7e8fNv/b3T7u3bt2FqaoobN25ALBbD19cXIpFIZr2BypNGRT8Jevz4MYDKvwWxWIwXL17A29tbJvyWlZUhIiICrq6uCtvaxwHoNa5evQonJyc4ODhg7969AIBevXrB0tISJ06cEA50X375Ja5du6bQB/7x48dj2rRpAP5quRo9ejT8/f1lljt27BicnZ2FZYHKliAnJyd069at/gquZ6WlpejUqRMiIyNlpkskEri5uSEkJERmmqKr2tm1uLgYERERMo8/WLp0qUwImjNnDpo2bSqzjKJLS0uDubk5RCIRoqKiZOZVDUEODg4IDg6GSCTCn3/+KZ9ia4n0c5ceAIHKZz9J7/D09PTEgAEDhHnSEDR58mSZMKjoVq1aBXt7e5kQk56ejlatWgn9/goLC+Hk5ARHR0csWLAAmzdvxtixY6Gnp6fQ/QC3bt2KPn36VPtb7tWrF/7zn/8A+OsYUVFRgZ49e8LX17fe66wNPBbYK0ybNo2ioqJIW1ubsrKy6Ouvv6Zdu3bR0aNHydramgICAmjYsGHk7u5OZ8+eJRsbG1JTU1PYsY/69etHc+bMISKiZ8+eCdMLCwuJ6K8xnTw8POjTTz+ln376iZ49e0YAqGnTpjR//nzKz88XBkNVZPj/cW0qKiqopKSEiIg0NTWpV69edPDgQdq5c6ewrEgkIlNTUzIwMBDGxHmfB/97UyKRiHbt2kVDhw4lFxcX2rlzJ2VmZgrzJ02aREuWLKEZM2bQmjVr6KuvvqK0tDRq0qSJHKuuPeXl5dS0aVNSV1cnQ0NDunnzJp07d06Yr6mpSYMGDaItW7ZQ27Zt6eHDh5SSkkJ2dnZyrPrdiUQievz4MXl7e9PcuXMpMTGRPDw8qKCggIiIwsPDKS0tjSZOnEhElduBiOjx48ekq6tLFRUVSjH2V9++famwsJA+++wzevDgARERPX/+nDQ0NKhZs2YkFoupcePGdPToUbKzs6NffvmFFixYQHl5eXTq1Cnq2LGjnNfg35N+bkVFRVRWVkbR0dF0/fp1IiJ6+fIl6evr07Vr14iISF1dncRiMamrq1P//v0pMzOTysrK5Fb7W5Nn+npfbdy4EQYGBrh48SKePHmCnJwc9OvXDy4uLkhMTARQ+byDsWPHYvz48UIzsSK2AP29tWLTpk3o378/srOzERcXB5FIhJMnT8os8/PPP6NLly4oKCgQpi1atAjW1tYyZ46KqOqdLiNHjkS7du0wdepUHD58GKWlpfD29kaXLl0wadIk/PLLLwgJCYGenp5CXO/+N86fPw89PT0EBwfjiy++gIaGBiZNmiTz/B8AmDdvHpo0aaLwn/urFBYWIiUlBW3atMHw4cOrPfBTStoHQpFJ1yEjIwOzZ8+GmZkZtLS0sG3bNmGZgoICrFixAq1atYKnpyemT5+OgIAANGrUCFeuXJFX6XXi7t27aN26Nbp3747c3FycPn0atra2Nfb3evHiBQoLC2UuCyqaqn/bmzdvhqenJ4YNGyZ8rqmpqdDW1sbEiRNljnX+/v4YMWKEQj7rigNQDSIjI9G9e3eIxWLhg87KykLnzp2FZ/9ISecrSyfIVatWoWvXrvjss8+QnZ2NSZMmQV9fH3v37sW9e/fw9OlT9O3bF97e3jLhaefOnQofAqTrs2vXLujo6GD69OnYsmULevXqhVatWgnr/80336BLly6wtrZGjx49cPnyZfkWXstu376NGTNmYN68ecK0VatWwczMDGFhYdVCkLJc9pJ+/snJydi2bRs2b94srOu5c+fQpk0b+Pj44PTp0wAqH4AYHR0t87OKavPmzTAyMhKeYZOUlASRSAQjIyOZvwOgMhQeP34c3t7e6NOnDz7++GOlCz9S0hDUr18/JCYmwtHREcuXL0d8fDyOHj2KhIQEbNiwAVevXpV3qe/kyJEjaN68ucxnvWnTJiEESe/o3LFjB7S1tdG7d2988cUXCAgIQOPGjRX2jk8OQFVId2LfffcdXFxchDtZpMn2yJEj0NHRgaenJ7Zv3y63OutabGwsevTogU8++QR//PEHwsLCoK2tDTMzM9jZ2cHR0VHhn3EktXfvXuHuFYlEgocPH8LDw0N4vkdxcTGaN2+O0NDQai18eXl57+XTTd/F8+fP4eLigg8++AAREREy81auXAlTU1NERkbKPNxR0Q/+VcXHx8Pc3Byurq7w8PCAhoYGdu/eDaBy0Nv27dujS5cu6NevH3R0dF7ZIqQopJ/d0aNH0bVrV9jb2+Phw4fIz89HYmIi5s2bB1tb22p9oKpSxDP/V5Fujxs3bggndHfv3kWbNm2EwT4dHBxgb28PZ2dnWFtbo23btrhx44Y8y35nN27cwNSpU9G+fXssWLBAmL5p0yb06tULw4YNw7Vr1wBUPg7D398fw4YNg7+/v0KHXw5ANUhNTYW6urpwdieVlJSEESNGoHfv3vDy8lKKOx2qqnog27BhAzw8PODr64snT54gJSUFcXFxiIuLU/hnHEnl5uaiVatWQid2oPLOD2dnZ9y4cQN37tyBqakpxowZI/xMUlIS0tLS5FVyvbh06RKsra3RrVu3aju31atXQ0tLCzNnzlT4z//vzp8/j6ZNm+LHH38EUDnEjUgkwnfffSeE3+TkZERERCAkJEThOzwDlc9uASq/+2fOnEH37t2FEARUPvsoKioKtra2MgP+bt26VegYrSwBWLoeO3bsgJ2dHWbMmCEM4nn37l04OzujQ4cOSE9PR3l5OcrLy4WbBBTVokWLhJO4jIwMfPvtt2jbtu0rQ5D0ZFEaehU9/HIAeoWNGzdCQ0MD06ZNw4ULF5Ceno7Bgwdjzpw5uHbtGkQikcxI38ri7yGoe/fu8PX1Fc74/z4chKK7ePEiXF1dERgYiCtXrqCkpAR2dnZYs2YN2rRpg8DAQGFd7969i5EjRyrd8z1qkpKSAkdHRwQFBVVr3l+/fr3CP/fp76PaA0BcXJxwS/OdO3dgZmaGcePGCfOll/oqKioUsr/f30kHt5Xe1SSRSHDq1Cl0794dtra2wuWwjIwMREdHw9bWFqNHj0ZkZCREIpFSPudn//790NbWxurVq4UQKHX37l1YWVnB09NTeBioIktLS0OPHj1kui6kpaW9NgT5+PjI3OGm6OGXA9BrxMfHw9DQEGZmZjA1NYWTkxNKSkpw7949WFtbC2lY2fw9BPXo0UMmBCmbS5cuoVOnTvjqq6/w4MEDrFy5EiKRSOZBdwAQEREBe3t7pdj5vQnpdgkMDFSK1g6pmka1B4DFixejW7duSE9Ph4WFBYKCgoSgk5iYiAkTJijVJc+qg9tWPcGRhqB27doJISgzMxMrVqyAm5sbunbtqlRDfACV611aWorPP/8cU6ZMEaYBsid70qeBDxgwQOFPAisqKoSx606ePCn8/1UhaMuWLXBycsLnn3+uNFc/OAD9g6ysLJw5cwYnTpwQdoZhYWGwtbUVmkeVUdUDQ2xsLHr27Ilvv/0WpaWlCp/6a3Lp0iU4OjoiMDAQhw4dwtSpU6Guro6FCxdi4cKFGDduHHR1dZWuw/M/uXTpEtzc3DBy5EiF7+Qu9apR7f/44w94eHigSZMm+OKLLwD8dZNDaGgoPvnkEzx//rz+C65j0sFtpU85f1UIKi0thVgsxrNnz+RZbp1yd3dHaGhojfOkJz4ZGRkKfxm86j78wYMH8PT0RJs2bYSA/6oQtG3btmo3QSgyDkD/wtWrV/H555+jWbNmKnEgrPolmTp1Krp37640yb8mly5dgrOzM8aOHYsDBw5gxYoVaN++PTp37oxPP/1UoTv7vQtpMJA+6VtZ/H1U+5cvXyIwMBDGxsZYuXIlioqKkJWVhbCwMDRr1kzh7/R5laqD20o7/1cNQR06dFDak72qrTxFRUXw8vLCZ599JkyTLpOZmYmwsDClu+x36dIlBAUFIS4uDu7u7ujUqVO1ENS+fftq/WGVBQegN1ReXo5Lly5hypQpSrsjrIl0BxEdHQ0rKyulPvsDKvsEubi4IDAwEA8ePJB5IrIqU8b1r3rgX7x4MYDK9RwxYgQ6dOiAxo0bw93dHa1bt1aqSz7/NLht1RD0+++/w97eHp07d1aKfk9S0m0gfZq/9Pk9u3fvlgnFUuHh4XBxcRFaDpXF4sWL4ezsjAsXLuDUqVPo2LEjnJ2dZUJQSEgIXF1d8fjxY6Vr/ecA9C8peq/3tyGRSPDLL78o9OPd/41Lly7B1dUVvr6+QthVti++KvqnA/+iRYsAVLYEJScnIzY2FqdOnUJWVpbcaq4r/zS4rTQEicVinD17Fnfv3pVrvbVJ+newb98+DB06FJ6envDx8RFa9ZctWwaRSIThw4fD398ffn5+0NPTU4oQLF33qneude/eHV5eXgAqH4bo6OgoE4LS09OVLvhJcQBirAbKetlH1b3pqPbKShUHt63Jrl27hJHNJ06ciEGDBkFLS0t45tPhw4cxatQoDBkyBMHBwUp1E0BSUhJGjRqF/fv3A6js02RlZYX58+cDqHw0gpubG6ysrJSq039NOAAx9grKeNlHVf2bA7/0QKBM3nZw2/z8fKVr/SwrK8PAgQNlRjYvLi7G5MmToampKTzVWNrfUZmedyWRSDBmzBiIRCI0bdoUUVFRuHPnDubMmYOPP/4YqampkEgkSEpKQq9evZT2zl+pBvIei4yx95WWlpa8S2DvCP8/OG1ZWRlpa2uTh4cH+fj4UJMmTSgyMpIaN25MoaGhREQUEBBAJSUltHDhQgoKClKagV2J/hrcdvXq1ZSZmUkSiYR8fX2FdZw0aRIREX3zzTdUUlJC48aNo+DgYGratKk8y651u3btorS0NLp37x59+OGHRFT5N6KpqUmzZ8+mq1ev0vLly2nVqlWkplY5Vri6uro8S35nqDJAs0gkosDAQCoqKiJ7e3tKSEigvLw8qqiooD///JMOHjxIHTp0oN69e1OPHj1IR0dHztXXLR4NnjGmtFR9VHupCxcukL+/P7Vq1Yrc3NwoPT2dNmzYQBkZGcIykyZNopkzZ9Ly5cupQYMGShd+Ll68SKNHj6aWLVuSjY0N7d27l4qLi0kkEpFIJKJGjRqRsbExPX78mDQ0NKhBg8r2AWl4UFQikYiOHDlC69evJyIiFxcXatasGaWnp9ORI0fIwcGBiIhu3LhBU6dOpTNnzpCGhobShx8iDkCMMSXGB36i9PR02r17N4WHh9Pq1atp48aNtGzZMtqxYwetWbNGZluEhYVReno6NWvWTI4V177bt29TYmIijRkzhnx8fGjgwIH06NEjiomJodLSUiHkNGjQgJo0aULl5eUEQM5V1w6xWEznzp2joKAg8vf3pzNnztDy5cspOTmZli5dSuPGjaOYmBiaMGECmZiYUPPmzeVdcr0RQVk+ZcYYqyI9PZ02b95M2traFBYWRkREq1evprlz59KoUaMoODiYWrZsKSz/9OlTpWv5KSgooD59+tC9e/coKCiI5syZI8z74YcfaN68efTFF1/QV199Ra1atSIi2UsmykC6DTIyMsjPz4+WLFlCFRUVFB4eTsePHyddXV3y8vKimzdv0o4dO+js2bPUvn17eZdd61JTU2natGlUVFRErq6uNGDAAFqzZg1988035O7uTkREz549IwMDA/kWWp/k2P+IMcbqhKqPal+Vqg5uW5V0Gzg6OuLixYsAKh90uGnTJnz22Wdwc3PDiBEjhA7Qyio3NxebN2+Go6MjGjVqhFatWiEyMlLeZckNByDGmFLiA/9flH1w2zeRkpICBwcHBAYGVhvHsbi4WKWe8VZWVobQ0FBoaGjA0NAQBQUF8i5JLvgSGGNMaaWmplJAQAC5ubnRxIkTZS5t/PTTT9SzZ0+ytraWY4X15/LlyxQYGEidOnWi0NBQsrOzk3dJ9a7qNpg8ebJSXur6J6hyifPQoUNkbW0tcylYlXAAYowpNT7w/+Xy5csUHBxMVlZWFBUVRba2tvIuqd7xNlC+fl5vi+8CY4wpNScnJ1q/fj2lpqbSrFmz6MaNG/IuSW6cnJxo5cqVlJOTQ/r6+vIuRy54Gyj+rf21hVuAGGMq4fz58zRt2jTatm0btWjRQt7lyFVpaanKP+iTtwHjAMQYUxl80GOMSXEAYowxxpjK4T5AjDHGGFM5HIAYY4wxpnI4ADHGGGNM5XAAYowxxpjK4QDEGGOMMZXDAYgxxhhjKocDEGOMMcZUDgcgxhhjjKkcDkCMMaUlFotJIpHIuwzG2HuIAxBjrF5ZWlrS0qVLZaY5OjpSdHQ0AaDo6GiysLAgTU1NMjExoYkTJwrLvXz5kqZOnUqmpqbUqFEj6ty5Mx07dkyYHxsbSwYGBpSYmEh2dnakqalJ9+/fJ0tLS5o7dy6NHj2adHV1ycLCgn788UeZGr799luysbEhHR0dsrKyounTp1N5ebkwPzo6mhwdHWnDhg1kYWFBjRs3pvHjx5NYLKaFCxeSsbExGRoa0pw5c2Te99mzZxQYGEjNmzcnPT096t27N6WkpNTeBmWMvZUG8i6AMcakduzYQUuWLKHt27dT+/btKTc3VyYshISE0LVr12j79u1kYmJCCQkJNGDAALpy5QpZW1sTEVFxcTEtWLCA1q9fT82aNSNDQ0MiIoqJiaFZs2ZRREQExcfH07hx48jDw4Patm1LRES6uroUGxtLJiYmdOXKFRozZgzp6urSN998I/z+9PR0+u233ygpKYnS09Pp448/pjt37pCNjQ0dP36cTp8+TaNHjyYvLy/q3LkzERH5+PiQtrY2/fbbb6Svr09r166lPn360K1bt6hp06b1tWkZY38HxhirRy1btsSSJUtkpnXs2BFRUVGIiYmBjY0NysrKqv1cRkYG1NXVkZ2dLTO9T58+CA8PBwBs3LgRRITk5ORqv3PUqFHCa4lEAkNDQ6xevfqVdX7//fdwdnYWXkdFRUFHRwcFBQXCtP79+8PS0hJisViY1rZtW8ybNw8AcPLkSejp6aG0tFTmvVu3bo21a9e+8nczxuoetwAxxt4bPj4+tHTpUrKysqIBAwbQoEGDaMiQIdSgQQO6cuUKicVisrGxkfmZly9fUrNmzYTXDRs2JAcHh2rvXXWaSCQiY2NjevjwoTDt559/puXLl1N6ejoVFRVRRUUF6enpybyHpaUl6erqCq+NjIxIXV2d1NTUZKZJ3zclJYWKiopk6iMiKikpofT09H+zaRhjtYwDEGOsXqmpqREAmWnSvjbm5uZ08+ZNOnToEB08eJDGjx9P33//PR0/fpyKiopIXV2dLl68SOrq6jI/37hxY+H/2traJBKJqv1eDQ0NmdcikUjoIH3mzBny8/OjmTNnUv/+/UlfX5+2b99OMTEx//ger3vfoqIiatGihUw/JSkDA4Nq0xhj9YcDEGOsXjVv3pxycnKE1wUFBXT37l3htba2Ng0ZMoSGDBlCEyZMIFtbW7py5Qo5OTmRWCymhw8fUo8ePWq1ptOnT1PLli0pMjJSmJaRkfHO79upUyfKzc2lBg0akKWl5Tu/H2Os9nAAYozVq969e1NsbCwNGTKEDAwMaMaMGUKLTmxsLInFYurcuTPp6OjQ1q1bSVtbm1q2bEnNmjUjPz8/8vf3p5iYGHJycqJHjx7R4cOHycHBgQYPHvzWNVlbW9P9+/dp+/bt5OrqSnv37qWEhIR3XlcvLy/q2rUrDR06lBYuXEg2Njb04MED2rt3Lw0bNoxcXFze+Xcwxt4O3wbPGKtX4eHh5OHhQd7e3jR48GAaOnQotW7dmogqLwutW7eOunXrRg4ODnTo0CHavXu30Idm48aN5O/vT1OmTKG2bdvS0KFD6fz582RhYfFONX344YcUGhpKISEh5OjoSKdPn6bp06e/87qKRCLat28f9ezZk7788kuysbGhkSNHUkZGBhkZGb3z+zPG3p4If78YzxhjjDGm5LgFiDHGGGMqhwMQY4wxxlQOByDGGGOMqRwOQIwxxhhTORyAGGOMMaZyOAAxxhhjTOVwAGKMMcaYyuEAxBhjjDGVwwGIMcYYYyqHAxBjjDHGVA4HIMYYY4ypnP8D5Cl+Ow2FVokAAAAASUVORK5CYII=\n", - "text/plain": [ - "
" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "df2 = pd.DataFrame(response2.json)\n", - "df2 = df2.sort_values('username')\n", - "df2['upvotes_normalized'] = df2['total_upvotes']/df2['num_posts']\n", - "\n", - "ax = df.plot(x='username', y='upvotes_normalized', marker='o', color='green', label=\"Time 1\")\n", - "df2.plot(x='username', y='upvotes_normalized', marker='o', color='purple', ax=ax, label=\"Time 2\")\n", - "plt.xticks(rotation=45, ha='right')\n", - "plt.ylabel(\"Number of upvotes (normalized)\")\n", - "plt.show()" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "This plot shows how some users maintain relatively consistent social media impact between the two query snapshots, whereas other users grow or decline in their influence.\n", - "\n", - "## Learn more\n", - "\n", - "This tutorial showed you how to create a Kafka topic using a Python client for Kafka, send a simulated stream of data to Kafka using a data generator, and query and visualize results over time. For more information, see the following resources:\n", - "\n", - "* [Apache Kafka ingestion](https://druid.apache.org/docs/latest/development/extensions-core/kafka-ingestion.html)\n", - "* [Querying data](https://druid.apache.org/docs/latest/tutorials/tutorial-query.html)\n", - "* [Tutorial: Run with Docker](https://druid.apache.org/docs/latest/tutorials/docker.html)" - ] - } - ], - "metadata": { - "kernelspec": { - "display_name": "Python 3 (ipykernel)", - "language": "python", - "name": "python3" - }, - "language_info": { - "codemirror_mode": { - "name": "ipython", - "version": 3 - }, - "file_extension": ".py", - "mimetype": "text/x-python", - "name": "python", - "nbconvert_exporter": "python", - "pygments_lexer": "ipython3", - "version": "3.8.16" - }, - "vscode": { - "interpreter": { - "hash": "a4289e5b8bae5973a6609d90f7bc464162478362b9a770893a3c5c597b0b36e7" - } - } - }, - "nbformat": 4, - "nbformat_minor": 4 -} diff --git a/examples/quickstart/jupyter-notebooks/0-START-HERE.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/00-START-HERE.ipynb similarity index 91% rename from examples/quickstart/jupyter-notebooks/0-START-HERE.ipynb rename to examples/quickstart/jupyter-notebooks/notebooks/01-introduction/00-START-HERE.ipynb index 0fdbad297f5..0f89633c22f 100644 --- a/examples/quickstart/jupyter-notebooks/0-START-HERE.ipynb +++ b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/00-START-HERE.ipynb @@ -66,7 +66,7 @@ "topics and use a simple set of Python wrappers around the underlying REST API. The\n", "wrappers reside in the `druidapi` package within this directory. While the package\n", "can be used in any Python program, the key purpose, at present, is to support these\n", - "notebooks. See the [Introduction to the Druid Python API](Python_API_Tutorial.ipynb)\n", + "notebooks. See the [Introduction to the Druid Python API](01-druidapi-package-intro.ipynb)\n", "for an overview of the Python API." ] }, @@ -87,12 +87,12 @@ "notebook directly, such as with `wget`, or manually through your web browser. Note\n", "that if you save the file from your web browser, make sure to remove the `.txt` extension.\n", "\n", - "- [Introduction to the Druid REST API](api-tutorial.ipynb) walks you through some of the\n", + "- [Introduction to the Druid REST API](../04-api/00-getting-started.ipynb) walks you through some of the\n", " basics related to the Druid REST API and several endpoints.\n", - "- [Introduction to the Druid Python API](Python_API_Tutorial.ipynb) walks you through some of the\n", + "- [Introduction to the Druid Python API](01-druidapi-package-intro.ipynb) walks you through some of the\n", " basics related to the Druid API using the Python wrapper API.\n", - "- [Learn the basics of Druid SQL](sql-tutorial.ipynb) introduces you to the unique aspects of Druid SQL with the primary focus on the SELECT statement. \n", - "- [Ingest and query data from Apache Kafka](kafka-tutorial.ipynb) walks you through ingesting an event stream from Kafka." + "- [Learn the basics of Druid SQL](../03-query/00-using-sql-with-druidapi.ipynb) introduces you to the unique aspects of Druid SQL with the primary focus on the SELECT statement. \n", + "- [Ingest and query data from Apache Kafka](../02-ingestion/01-streaming-from-kafka.ipynb) walks you through ingesting an event stream from Kafka." ] }, { @@ -154,7 +154,7 @@ "name": "python", "nbconvert_exporter": "python", "pygments_lexer": "ipython3", - "version": "3.8.16" + "version": "3.11.4" } }, "nbformat": 4, diff --git a/examples/quickstart/jupyter-notebooks/Python_API_Tutorial.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb similarity index 99% rename from examples/quickstart/jupyter-notebooks/Python_API_Tutorial.ipynb rename to examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb index 851f1896db9..f3b104b14fd 100644 --- a/examples/quickstart/jupyter-notebooks/Python_API_Tutorial.ipynb +++ b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb @@ -217,16 +217,6 @@ "The above shows the list of datasources by default. You'll get an empty result if you have no datasources yet." ] }, - { - "cell_type": "code", - "execution_count": null, - "id": "616770ce", - "metadata": {}, - "outputs": [], - "source": [ - "display.tables()" - ] - }, { "cell_type": "markdown", "id": "7392e484", @@ -491,7 +481,7 @@ "source": [ "## Datasource Client\n", "\n", - "The Datasource client lets you perform operations on datasource objects. The SQL layer allows you to get metadata and do queries. The datasource client works with the underlying segments. Explaining the full functionality is the topic of another notebook. For now, you can use the datasource client to clean up the datasource created above. The `True` argument asks for \"if exists\" semantics so you don't get an error if the datasource was alredy deleted." + "The Datasource client lets you perform operations on datasource objects. The SQL layer allows you to get metadata and do queries. The datasource client works with the underlying segments. Explaining the full functionality is the topic of another notebook. For now, you can use the datasource client to clean up the datasource created above. The `True` argument asks for \"if exists\" semantics so you don't get an error if the datasource was already deleted." ] }, { @@ -579,9 +569,7 @@ "cell_type": "code", "execution_count": null, "id": "9e42dfbc", - "metadata": { - "scrolled": true - }, + "metadata": {}, "outputs": [], "source": [ "rest_client.get_json('/status')" @@ -741,7 +729,7 @@ "name": "python", "nbconvert_exporter": "python", "pygments_lexer": "ipython3", - "version": "3.8.16" + "version": "3.11.4" } }, "nbformat": 4, diff --git a/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb new file mode 100644 index 00000000000..6a62dbbd197 --- /dev/null +++ b/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb @@ -0,0 +1,537 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Ingest and query data from Apache Kafka\n", + "\n", + "\n", + "\n", + "This tutorial introduces you to streaming ingestion in Apache Druid using the Apache Kafka event streaming platform.\n", + "Follow along to learn how to create and load data into a Kafka topic, start ingesting data from the topic into Druid, and query results over time. This tutorial assumes you have a basic understanding of Druid ingestion, querying, and API requests." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Table of contents\n", + "\n", + "* [Prerequisites](#Prerequisites)\n", + "* [Load Druid API client](#Load-Druid-API-client)\n", + "* [Create Kafka topic](#Create-Kafka-topic)\n", + "* [Load data into Kafka topic](#Load-data-into-Kafka-topic)\n", + "* [Start Druid ingestion](#Start-Druid-ingestion)\n", + "* [Query Druid datasource and visualize query results](#Query-Druid-datasource-and-visualize-query-results)\n", + "* [Learn more](#Learn-more)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Prerequisites\n", + "\n", + "This tutorial works with Druid 25.0.0 or later.\n", + "\n", + "Launch this tutorial and all prerequisites using the `all-services` profile of the Docker Compose file for Jupyter-based Druid tutorials. For more information, see [Docker for Jupyter Notebook tutorials](https://druid.apache.org/docs/latest/tutorials/tutorial-jupyter-docker.html).\n", + "\n", + "If you do not use the Docker Compose environment, you need the following:\n", + "* A running Druid instance.\n", + " * Update the `druid_host` variable to point to your Router endpoint. For example, `druid_host = \"http://localhost:8888\"`.\n", + " * Update the `rest_client` variable to point to your Coordinator endpoint. For example, `\"http://localhost:8081\"`.\n", + "* A running Kafka cluster.\n", + " * Update the Kafka bootstrap servers to point to your servers. For example, `bootstrap_servers=[\"localhost:9092\"]`.\n", + "* The following Python packages:\n", + " * `druidapi`, a Python client for Apache Druid\n", + " * `DruidDataDriver`, a data generator\n", + " * `kafka`, a Python client for Apache Kafka\n", + " * `pandas`, `matplotlib`, and `seaborn` for data visualization\n" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Load Druid API client" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "To start the tutorial, run the following cell. It imports the required Python packages and defines a variable for the Druid client, and another for the SQL client used to run SQL commands." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import druidapi\n", + "import json\n", + "\n", + "# druid_host is the hostname and port for your Druid deployment. \n", + "# In the Docker Compose tutorial environment, this is the Router\n", + "# service running at \"http://router:8888\".\n", + "# If you are not using the Docker Compose environment, edit the `druid_host`.\n", + "\n", + "druid_host = \"http://router:8888\"\n", + "druid_host\n", + "\n", + "druid = druidapi.jupyter_client(druid_host)\n", + "display = druid.display\n", + "sql_client = druid.sql\n", + "\n", + "# Create a rest client for native JSON ingestion for streaming data\n", + "rest_client = druidapi.rest.DruidRestClient(\"http://coordinator:8081\")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Create Kafka topic" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "This notebook relies on the Python client for the Apache Kafka. Import the Kafka producer and consumer modules, then create a Kafka client. You use the Kafka producer to create and publish records to a new topic named `social_media`." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "from kafka import KafkaProducer\n", + "from kafka import KafkaConsumer\n", + "\n", + "# Kafka runs on kafka:9092 in multi-container tutorial application\n", + "producer = KafkaProducer(bootstrap_servers='kafka:9092')\n", + "topic_name = \"social_media\"" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Create the `social_media` topic and send a sample event. The `send()` command returns a metadata descriptor for the record." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "event = {\n", + " \"__time\": \"2023-01-03T16:40:21.501\",\n", + " \"username\": \"willow\",\n", + " \"post_title\": \"This title is required\",\n", + " \"views\": 15284,\n", + " \"upvotes\": 124,\n", + " \"comments\": 21,\n", + " \"edited\": \"True\"\n", + "}\n", + "\n", + "producer.send(topic_name, json.dumps(event).encode('utf-8'))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "To verify that the Kafka topic stored the event, create a consumer client to read records from the Kafka cluster, and get the next (only) message:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "consumer = KafkaConsumer(topic_name, bootstrap_servers=['kafka:9092'], auto_offset_reset='earliest',\n", + " enable_auto_commit=True)\n", + "\n", + "print(next(consumer).value.decode('utf-8'))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Load data into Kafka topic" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Instead of manually creating events to send to the Kafka topic, use a data generator to simulate a continuous data stream. This tutorial makes use of Druid Data Driver to simulate a continuous data stream into the `social_media` Kafka topic. To learn more about the Druid Data Driver, see the Druid Summit talk, [Generating Time centric Data for Apache Druid](https://www.youtube.com/watch?v=3zAOeLe3iAo).\n", + "\n", + "In this notebook, you use a background process to continuously load data into the Kafka topic.\n", + "This allows you to keep executing commands in this notebook while data is constantly being streamed into the topic." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Run the following cells to load sample data into the `social_media` Kafka topic:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import multiprocessing as mp\n", + "from datetime import datetime\n", + "import DruidDataDriver" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def run_driver():\n", + " DruidDataDriver.simulate(\"kafka_docker_config.json\", None, None, \"REAL\", datetime.now())\n", + " \n", + "mp.set_start_method('fork')\n", + "ps = mp.Process(target=run_driver)\n", + "ps.start()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Start Druid ingestion" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Now that you have a new Kafka topic and data being streamed into the topic, you ingest the data into Druid by submitting a Kafka ingestion spec.\n", + "The ingestion spec describes the following:\n", + "* where to source the data to ingest (in `spec > ioConfig`),\n", + "* the datasource to ingest data into (in `spec > dataSchema > dataSource`), and\n", + "* what the data looks like (in `spec > dataSchema > dimensionsSpec`).\n", + "\n", + "Other properties control how Druid aggregates and stores data. For more information, see the Druid documenation:\n", + "* [Apache Kafka ingestion](https://druid.apache.org/docs/latest/development/extensions-core/kafka-ingestion.html)\n", + "* [Ingestion spec reference](https://druid.apache.org/docs/latest/ingestion/ingestion-spec.html)\n", + "\n", + "Run the following cells to define and view the Kafka ingestion spec." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "kafka_ingestion_spec = \"{\\\"type\\\": \\\"kafka\\\",\\\"spec\\\": {\\\"ioConfig\\\": {\\\"type\\\": \\\"kafka\\\",\\\"consumerProperties\\\": {\\\"bootstrap.servers\\\": \\\"kafka:9092\\\"},\\\"topic\\\": \\\"social_media\\\",\\\"inputFormat\\\": {\\\"type\\\": \\\"json\\\"},\\\"useEarliestOffset\\\": true},\\\"tuningConfig\\\": {\\\"type\\\": \\\"kafka\\\"},\\\"dataSchema\\\": {\\\"dataSource\\\": \\\"social_media\\\",\\\"timestampSpec\\\": {\\\"column\\\": \\\"__time\\\",\\\"format\\\": \\\"iso\\\"},\\\"dimensionsSpec\\\": {\\\"dimensions\\\": [\\\"username\\\",\\\"post_title\\\",{\\\"type\\\": \\\"long\\\",\\\"name\\\": \\\"views\\\"},{\\\"type\\\": \\\"long\\\",\\\"name\\\": \\\"upvotes\\\"},{\\\"type\\\": \\\"long\\\",\\\"name\\\": \\\"comments\\\"},\\\"edited\\\"]},\\\"granularitySpec\\\": {\\\"queryGranularity\\\": \\\"none\\\",\\\"rollup\\\": false,\\\"segmentGranularity\\\": \\\"hour\\\"}}}}\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "print(json.dumps(json.loads(kafka_ingestion_spec), indent=4))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Send the spec to Druid to start the streaming ingestion from Kafka:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "headers = {\n", + " 'Content-Type': 'application/json'\n", + "}\n", + "\n", + "rest_client.post(\"/druid/indexer/v1/supervisor\", kafka_ingestion_spec, headers=headers)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "A `200` response indicates that the request was successful. You can view the running ingestion task and the new datasource in the web console at http://localhost:8888/unified-console.html." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Query Druid datasource and visualize query results" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "You can now query the new datasource called `social_media`. In this section, you also visualize query results using the Matplotlib and Seaborn visualization libraries. Run the following cell import these packages." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import pandas as pd\n", + "import matplotlib\n", + "import matplotlib.pyplot as plt\n", + "import seaborn as sns" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Run a simple query to view a subset of rows from the new datasource:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "sql = '''\n", + "SELECT * FROM social_media LIMIT 5\n", + "'''\n", + "display.sql(sql)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "In this social media scenario, each incoming event represents a post on social media, for which you collect the timestamp, username, and post metadata. You are interested in analyzing the total number of upvotes for all posts, compared between users. Preview this data with the following query:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "sql = '''\n", + "SELECT\n", + " COUNT(post_title) as num_posts,\n", + " SUM(upvotes) as total_upvotes,\n", + " username\n", + "FROM social_media\n", + "GROUP BY username\n", + "ORDER BY num_posts\n", + "'''\n", + "\n", + "response = sql_client.sql_query(sql)\n", + "response.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Visualize the total number of upvotes per user using a line plot. You sort the results by username before plotting because the order of users may vary as new results arrive." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "df = pd.DataFrame(response.json)\n", + "df = df.sort_values('username')\n", + "\n", + "df.plot(x='username', y='total_upvotes', marker='o')\n", + "plt.xticks(rotation=45, ha='right')\n", + "plt.ylabel(\"Total number of upvotes\")\n", + "plt.gca().get_legend().remove()\n", + "plt.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "The total number of upvotes likely depends on the total number of posts created per user. To better assess the relative impact per user, you compare the total number of upvotes (line plot) with the total number of posts." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "matplotlib.rc_file_defaults()\n", + "ax1 = sns.set_style(style=None, rc=None )\n", + "\n", + "fig, ax1 = plt.subplots()\n", + "plt.xticks(rotation=45, ha='right')\n", + "\n", + "\n", + "sns.lineplot(\n", + " data=df, x='username', y='total_upvotes',\n", + " marker='o', ax=ax1, label=\"Sum of upvotes\")\n", + "ax1.get_legend().remove()\n", + "\n", + "ax2 = ax1.twinx()\n", + "sns.barplot(data=df, x='username', y='num_posts',\n", + " order=df['username'], alpha=0.5, ax=ax2, log=True,\n", + " color=\"orange\", label=\"Number of posts\")\n", + "\n", + "\n", + "# ask matplotlib for the plotted objects and their labels\n", + "lines, labels = ax1.get_legend_handles_labels()\n", + "lines2, labels2 = ax2.get_legend_handles_labels()\n", + "ax2.legend(lines + lines2, labels + labels2, bbox_to_anchor=(1.55, 1))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "You should see a correlation between total number of upvotes and total number of posts. In order to track user impact on a more equal footing, normalize the total number of upvotes relative to the total number of posts, and plot the result:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "df['upvotes_normalized'] = df['total_upvotes']/df['num_posts']\n", + "\n", + "df.plot(x='username', y='upvotes_normalized', marker='o', color='green')\n", + "plt.xticks(rotation=45, ha='right')\n", + "plt.ylabel(\"Number of upvotes (normalized)\")\n", + "plt.gca().get_legend().remove()\n", + "plt.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "You've been working with data taken at a single snapshot in time from when you ran the last query. Run the same query again, and store the output in `response2`, which you will compare with the previous results:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "response2 = sql_client.sql_query(sql)\n", + "response2.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Normalizing the data also helps you evaluate trends over time more consistently on the same plot axes. Plot the normalized data again, this time alongside the results from the previous snapshot:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "df2 = pd.DataFrame(response2.json)\n", + "df2 = df2.sort_values('username')\n", + "df2['upvotes_normalized'] = df2['total_upvotes']/df2['num_posts']\n", + "\n", + "ax = df.plot(x='username', y='upvotes_normalized', marker='o', color='green', label=\"Time 1\")\n", + "df2.plot(x='username', y='upvotes_normalized', marker='o', color='purple', ax=ax, label=\"Time 2\")\n", + "plt.xticks(rotation=45, ha='right')\n", + "plt.ylabel(\"Number of upvotes (normalized)\")\n", + "plt.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "This plot shows how some users maintain relatively consistent social media impact between the two query snapshots, whereas other users grow or decline in their influence.\n", + "\n", + "## Learn more\n", + "\n", + "This tutorial showed you how to create a Kafka topic using a Python client for Kafka, send a simulated stream of data to Kafka using a data generator, and query and visualize results over time. For more information, see the following resources:\n", + "\n", + "* [Apache Kafka ingestion](https://druid.apache.org/docs/latest/development/extensions-core/kafka-ingestion.html)\n", + "* [Querying data](https://druid.apache.org/docs/latest/tutorials/tutorial-query.html)\n", + "* [Tutorial: Run with Docker](https://druid.apache.org/docs/latest/tutorials/docker.html)" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.11.4" + }, + "vscode": { + "interpreter": { + "hash": "a4289e5b8bae5973a6609d90f7bc464162478362b9a770893a3c5c597b0b36e7" + } + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/DruidDataDriver.py b/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/DruidDataDriver.py new file mode 100644 index 00000000000..5acd25210be --- /dev/null +++ b/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/DruidDataDriver.py @@ -0,0 +1,1133 @@ +# 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. +# +# DruidDataDriver - generates JSON records as a workload for Apache Druid. +# + +import argparse +import dateutil.parser +from datetime import datetime, timedelta +import json +import numpy as np +import random +import re +from sortedcontainers import SortedList +import string +import sys +import threading +import time + +############################################################################ +# +# DruidDataDriver simulates Druid workloads by producing JSON records. +# Use a JSON config file to describe the characteristics of the workload +# you want to simulate. +# +# Run the program as follows: +# python DruidDataDriver.py +# Options include: +# -n +# -t +# +# See the associated documentation for the format of the config file. +# +############################################################################ + + +class FutureEvent: + def __init__(self, t): + self.t = t + self.name = threading.current_thread().name + self.event = threading.Event() + def get_time(self): + return self.t + def get_name(self): + return self.name + def __lt__(self, other): + return self.t < other.t + def __eq__(self, other): + return self.t == other.t + def __le__(self, other): + return self.t <= other.t + def __gt__(self, other): + return self.t > other.t + def __ge__(self, other): + return self.t >= other.t + + def __str__(self): + return 'FutureEvent('+self.name+', '+str(self.t)+')' + def pause(self): + #print(self.name+" pausing") + self.event.clear() + self.event.wait() + def resume(self): + #print(self.name+" resuming") + self.event.set() + +class Clock: + future_events = SortedList() + active_threads = 0 + lock = threading.Lock() + sleep_lock = threading.Lock() + + def __init__(self, time_type, start_time = datetime.now()): + self.sim_time = start_time + self.time_type = time_type + + def __str__(self): + s = 'Clock(time='+str(self.sim_time) + for e in self.future_events: + s += ', '+str(e) + s += ')' + return s + + def activate_thread(self): + if self.time_type == 'SIM': + self.lock.acquire() + self.active_threads += 1 + self.lock.release() + + def deactivate_thread(self): + if self.time_type == 'SIM': + self.lock.acquire() + self.active_threads -= 1 + self.lock.release() + + def end_thread(self): + if self.time_type == 'SIM': + self.lock.acquire() + self.active_threads -= 1 + if len(self.future_events) > 0: + self.remove_event().resume() + self.lock.release() + + def release_all(self): + if self.time_type == 'SIM': + self.lock.acquire() + #print('release_all - active_threads = '+str(self.active_threads)) + for e in self.future_events: + e.resume() + self.lock.release() + + def add_event(self, future_t): + this_event = FutureEvent(future_t) + self.future_events.add(this_event) + #print('add_event (after) '+threading.current_thread().name+' - '+str(self)) + return this_event + + def remove_event(self): + #print('remove_event (before) '+threading.current_thread().name+' - '+str(self)) + next_event = self.future_events[0] + self.future_events.remove(next_event) + return next_event + + def pause(self, event): + self.active_threads -= 1 + self.lock.release() + event.pause() + self.lock.acquire() + self.active_threads += 1 + + def resume(self, event): + event.resume() + + def now(self): + if self.time_type == 'SIM': + t = self.sim_time + else: + t = datetime.now() + return t + + def sleep(self, delta): + if self.time_type == 'SIM': # Simulated time + self.lock.acquire() + #print(threading.current_thread().name+" begin sleep "+str(self.sim_time)+" + "+str(delta)) + this_event = self.add_event(self.sim_time + timedelta(seconds=delta)) + #print(threading.current_thread().name+" active threads "+str(self.active_threads)) + if self.active_threads == 1: + next_event = self.remove_event() + if str(this_event) != str(next_event): + self.resume(next_event) + #print(threading.current_thread().name+" start pause if") + self.pause(this_event) + #print(threading.current_thread().name+" end pause if") + else: + #print(threading.current_thread().name+" start pause else") + self.pause(this_event) + #print(threading.current_thread().name+" end pause else") + self.sim_time = this_event.get_time() + #print(threading.current_thread().name+" end sleep "+str(self.sim_time)) + self.lock.release() + + else: # Real time + time.sleep(delta) + + +# +# Set up the target +# + +class PrintStdout: + lock = threading.Lock() + def print(self, record): + with self.lock: + print(str(record)) + sys.stdout.flush() + def __str__(self): + return '#printStdout()' + +class PrintFile: + f = None + def __init__(self, file_name): + self.file_name = file_name + self.f = open(file_name, 'w') + def __del__(self): + if self.f != None: + self.f.close() + def __str__(self): + return 'PrintFile(file_name='+self.file_name+')' + def print(self, record): + self.f.write(str(record)+'\n') + self.f.flush() + +class PrintKafka: + producer = None + topic = None + def __init__(self, endpoint, topic, security_protocol, compression_type): + from kafka import KafkaProducer + + #print('PrintKafka('+str(endpoint)+', '+str(topic)+', '+str(security_protocol)+', '+str(compression_type)+')') + self.endpoint = endpoint + self.producer = KafkaProducer(bootstrap_servers=endpoint, security_protocol=security_protocol, compression_type=compression_type, value_serializer=lambda v: json.dumps(v).encode('utf-8')) + self.topic = topic + def __str__(self): + return 'PrintKafka(endpoint='+self.endpoint+', topic='+self.topic+')' + def print(self, record): + self.producer.send(self.topic, json.loads(str(record))) + +class PrintConfluent: + producer = None + topic = None + username = None + password = None + def __init__(self, servers, topic, username, password): + from confluent_kafka import Producer + + #print('PrintKafka('+str(endpoint)+', '+str(topic)+', '+str(security_protocol)+', '+str(compression_type)+')') + self.servers = servers + self.producer = Producer({ + 'bootstrap.servers': servers, + 'sasl.mechanisms': 'PLAIN', + 'security.protocol': 'SASL_SSL', + 'sasl.username': username, + 'sasl.password': password + }) + self.topic = topic + self.username = username + self.password = password + def __str__(self): + return 'PrintConfluent(servers='+self.servers+', topic='+self.topic+', username='+self.username+', password='+self.password+')' + def print(self, record): + print('producing '+str(record)) + self.producer.produce(topic=self.topic, value=str(record)) + self.producer.flush() + + +# +# Handle distributions +# + +class DistConstant: + def __init__(self, value): + self.value = value + def __str__(self): + return 'DistConstant(value='+str(self.value)+')' + def get_sample(self): + return self.value + +class DistUniform: + def __init__(self, min_value, max_value): + self.min_value = min_value + self.max_value = max_value + def __str__(self): + return 'DistUniform(min_value='+str(self.min_value)+', max_value='+str(self.max_value)+')' + def get_sample(self): + return np.random.uniform(self.min_value, self.max_value+1) + +class DistExponential: + def __init__(self, mean): + self.mean = mean + def __str__(self): + return 'DistExponential(mean='+str(self.mean)+')' + def get_sample(self): + return np.random.exponential(scale = self.mean) + +class DistNormal: + def __init__(self, mean, stddev): + self.mean = mean + self.stddev = stddev + def __str__(self): + return 'DistNormal(mean='+str(self.mean )+', stddev='+str(self.stddev)+')' + def get_sample(self): + return np.random.normal(self.mean, self.stddev) + +def parse_distribution(desc): + dist_type = desc['type'].lower() + dist_gen = None + if dist_type == 'constant': + value = desc['value'] + dist_gen = DistConstant(value) + elif dist_type == 'uniform': + min_value = desc['min'] + max_value = desc['max'] + dist_gen = DistUniform(min_value, max_value) + elif dist_type == 'exponential': + mean = desc['mean'] + dist_gen = DistExponential(mean) + elif dist_type == 'normal': + mean = desc['mean'] + stddev = desc['stddev'] + dist_gen = DistNormal(mean, stddev) + else: + print('Error: Unknown distribution "'+dist_type+'"') + exit() + return dist_gen + +def parse_timestamp_distribution(desc): + dist_type = desc['type'].lower() + dist_gen = None + if dist_type == 'constant': + value = dateutil.parser.isoparse(desc['value']).timestamp() + dist_gen = DistConstant(value) + elif dist_type == 'uniform': + min_value = dateutil.parser.isoparse(desc['min']).timestamp() + max_value = dateutil.parser.isoparse(desc['max']).timestamp() + dist_gen = DistUniform(min_value, max_value) + elif dist_type == 'exponential': + mean = dateutil.parser.isoparse(desc['mean']).timestamp() + dist_gen = DistExponential(mean) + elif dist_type == 'normal': + mean = desc[dateutil.parser.isoparse(desc['mean']).timestamp()] + stddev = desc['stddev'] + dist_gen = DistNormal(mean, stddev) + else: + print('Error: Unknown distribution "'+dist_type+'"') + exit() + return dist_gen + + +# +# Set up the dimensions for the emitters (see below) +# There is one element class for each dimension type. This code creates a list of +# elements and then runs through the list to create a single record. +# Notice that the get_json_field_string() method produces the JSON dimension +# field object based on the dimension configuration. +# The get_stochastic_value() method is like a private method used to get a random +# idividual value. +# + +class ElementNow: # The __time dimension + def __init__(self, global_clock): + self.global_clock = global_clock + def __str__(self): + return 'ElementNow()' + def get_json_field_string(self): + now = self.global_clock.now().isoformat()[:-3] + return '"__time":"'+now+'"' + +class ElementCounter: # The __time dimension + def __init__(self, desc): + self.name = desc['name'] + if 'percent_nulls' in desc.keys(): + self.percent_nulls = desc['percent_nulls'] / 100.0 + else: + self.percent_nulls = 0.0 + if 'percent_missing' in desc.keys(): + self.percent_missing = desc['percent_missing'] / 100.0 + else: + self.percent_missing = 0.0 + if 'start' in desc.keys(): + self.start = desc['start'] + else: + self.start = 0 + if 'increment' in desc.keys(): + self.increment = desc['increment'] + else: + self.increment = 1 + self.value = self.start + def __str__(self): + s = 'ElementCounter(name='+self.name + if self.start != 0: + s += ', '+str(self.start) + if self.increment != 1: + s += ', '+str(self.increment) + s += ')' + return s + + def get_stochastic_value(self): + v = self.value + self.value += self.increment + return v + + def get_json_field_string(self): + if random.random() < self.percent_nulls: + s = '"'+self.name+'": null' + else: + s = '"'+self.name+'":"'+str(self.get_stochastic_value())+'"' + return s + + def is_missing(self): + return random.random() < self.percent_missing + + +class ElementEnum: # enumeration dimensions + def __init__(self, desc): + self.name = desc['name'] + if 'percent_nulls' in desc.keys(): + self.percent_nulls = desc['percent_nulls'] / 100.0 + else: + self.percent_nulls = 0.0 + if 'percent_missing' in desc.keys(): + self.percent_missing = desc['percent_missing'] / 100.0 + else: + self.percent_missing = 0.0 + self.cardinality = desc['values'] + if 'cardinality_distribution' not in desc.keys(): + print('Element '+self.name+' specifies a cardinality without a cardinality distribution') + exit() + self.cardinality_distribution = parse_distribution(desc['cardinality_distribution']) + + def __str__(self): + return 'ElementEnum(name='+self.name+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')' + + def get_stochastic_value(self): + index = int(self.cardinality_distribution.get_sample()) + if index < 0: + index = 0 + if index >= len(self.cardinality): + index = len(self.cardinality)-1 + return self.cardinality[index] + + def get_json_field_string(self): + if random.random() < self.percent_nulls: + s = '"'+self.name+'": null' + else: + s = '"'+self.name+'":"'+str(self.get_stochastic_value())+'"' + return s + + def is_missing(self): + return random.random() < self.percent_missing + +class ElementVariable: # Variable dimensions + def __init__(self, desc): + self.name = desc['name'] + self.variable_name = desc['variable'] + + def __str__(self): + return 'ElementVariable(name='+self.name+', value='+self.variable_name+')' + + def get_json_field_string(self, variables): # NOTE: because of timing, this method has a different signature than the other elements + value = variables[self.variable_name] + return '"'+self.name+'":"'+str(value)+'"' + +# TODO: Refactor ElementBase and subclasses, and those element classes that don't inherit from ElementBase +class ElementBase: # Base class for the remainder of the dimensions + def __init__(self, desc): + self.name = desc['name'] + if 'percent_nulls' in desc.keys(): + self.percent_nulls = desc['percent_nulls'] / 100.0 + else: + self.percent_nulls = 0.0 + if 'percent_missing' in desc.keys(): + self.percent_missing = desc['percent_missing'] / 100.0 + else: + self.percent_missing = 0.0 + + self.cardinality_setting = desc['cardinality'] + self.cardinality_distribution = None + + if self.cardinality_setting == 0: + self.cardinality = None + + else: + self.cardinality = [] + if 'cardinality_distribution' not in desc.keys(): + print('Element '+self.name+' specifies a cardinality without a cardinality distribution') + exit() + self.cardinality_distribution = parse_distribution(desc['cardinality_distribution']) + self.init_cardinality() + + def init_cardinality(self): + for i in range(self.cardinality_setting): + value = None + while True: + value = self.get_stochastic_value() + if value not in self.cardinality: + break + self.cardinality.append(value) + + + def get_stochastic_value(self): + pass + + def get_json_field_string(self): + if random.random() < self.percent_nulls: + s = '"'+self.name+'": null' + else: + if self.cardinality is None: + value = self.get_stochastic_value() + else: + index = int(self.cardinality_distribution.get_sample()) + if index < 0: + index = 0 + if index >= len(self.cardinality): + index = len(self.cardinality)-1 + value = self.cardinality[index] + s = '"'+self.name+'":'+str(value) + return s + + def is_missing(self): + return random.random() < self.percent_missing + + +class ElementString(ElementBase): + + def __init__(self, desc): + self.length_distribution = parse_distribution(desc['length_distribution']) + if 'chars' in desc: + self.chars = desc['chars'] + else: + self.chars = string.printable + super().__init__(desc) + + def __str__(self): + return 'ElementString(name='+self.name+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+', chars='+self.chars+')' + + def get_stochastic_value(self): + length = int(self.length_distribution.get_sample()) + return ''.join(random.choices(list(self.chars), k=length)) + + def get_json_field_string(self): + if random.random() < self.percent_nulls: + s = '"'+self.name+'": null' + else: + if self.cardinality is None: + value = self.get_stochastic_value() + else: + index = int(self.cardinality_distribution.get_sample()) + if index < 0: + index = 0 + if index >= len(self.cardinality): + index = len(self.cardinality)-1 + value = self.cardinality[index] + s = '"'+self.name+'":"'+str(value)+'"' + return s + +class ElementInt(ElementBase): + def __init__(self, desc): + self.value_distribution = parse_distribution(desc['distribution']) + super().__init__(desc) + + def __str__(self): + return 'ElementInt(name='+self.name+', value_distribution='+str(self.value_distribution)+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')' + + def get_stochastic_value(self): + return int(self.value_distribution.get_sample()) + +class ElementFloat(ElementBase): + def __init__(self, desc): + self.value_distribution = parse_distribution(desc['distribution']) + if 'precision' in desc: + self.precision = desc['precision'] + else: + self.precision = None + super().__init__(desc) + + def __str__(self): + return 'ElementFloat(name='+self.name+', value_distribution='+str(self.value_distribution)+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')' + + def get_stochastic_value(self): + return float(self.value_distribution.get_sample()) + + def get_json_field_string(self): + if random.random() < self.percent_nulls: + s = '"'+self.name+'": null' + else: + if self.cardinality is None: + value = self.get_stochastic_value() + else: + index = int(self.cardinality_distribution.get_sample()) + if index < 0: + index = 0 + if index >= len(self.cardinality): + index = len(self.cardinality)-1 + value = self.cardinality[index] + if self.precision is None: + s = '"'+self.name+'":'+str(value) + else: + format = '%.'+str(self.precision)+'f' + s = '"'+self.name+'":'+str(format%value) + return s + +class ElementTimestamp(ElementBase): + def __init__(self, desc): + super().__init__(desc) + + def __str__(self): + return 'ElementTimestamp(name='+self.name+', value_distribution='+str(self.value_distribution)+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')' + + def get_stochastic_value(self): + return datetime.fromtimestamp(self.value_distribution.get_sample()).isoformat()[:-3] + + def get_json_field_string(self): + if random.random() < self.percent_nulls: + s = '"'+self.name+'": null' + else: + if self.cardinality is None: + value = self.get_stochastic_value() + else: + index = int(self.cardinality_distribution.get_sample()) + if index < 0: + index = 0 + if index >= len(self.cardinality): + index = len(self.cardinality)-1 + value = self.cardinality[index] + s = '"'+self.name+'":"'+str(value)+'"' + return s + + def is_missing(self): + return random.random() < self.percent_missing + +class ElementIPAddress(ElementBase): + def __init__(self, desc): + self.value_distribution = parse_distribution(desc['distribution']) + super().__init__(desc) + + def __str__(self): + return 'ElementIPAddress(name='+self.name+', value_distribution='+str(self.value_distribution)+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')' + + def get_stochastic_value(self): + value = int(self.value_distribution.get_sample()) + return str((value & 0xFF000000) >> 24)+'.'+str((value & 0x00FF0000) >> 16)+'.'+str((value & 0x0000FF00) >> 8)+'.'+str(value & 0x000000FF) + + def get_json_field_string(self): + if random.random() < self.percent_nulls: + s = '"'+self.name+'": null' + else: + if self.cardinality is None: + value = self.get_stochastic_value() + else: + index = int(self.cardinality_distribution.get_sample()) + if index < 0: + index = 0 + if index >= len(self.cardinality): + index = len(self.cardinality)-1 + value = self.cardinality[index] + s = '"'+self.name+'":"'+str(value)+'"' + return s + +class ElementObject(): + def __init__(self, desc): + self.name = desc['name'] + self.dimensions = get_variables(desc['dimensions']) + if 'percent_nulls' in desc.keys(): + self.percent_nulls = desc['percent_nulls'] / 100.0 + else: + self.percent_nulls = 0.0 + if 'percent_missing' in desc.keys(): + self.percent_missing = desc['percent_missing'] / 100.0 + else: + self.percent_missing = 0.0 + cardinality = desc['cardinality'] + if cardinality == 0: + self.cardinality = None + self.cardinality_distribution = None + else: + self.cardinality = [] + if 'cardinality_distribution' not in desc.keys(): + print('Element '+self.name+' specifies a cardinality without a cardinality distribution') + exit() + self.cardinality_distribution = parse_distribution(desc['cardinality_distribution']) + for i in range(cardinality): + value = None + while True: + value = self.get_instance() + if value not in self.cardinality: + break + self.cardinality.append(value) + + def __str__(self): + s = 'ElementObject(name='+self.name+', dimensions=[' + for e in self.dimensions: + s += ',' + str(e) + s += '])' + return s + + def get_instance(self): + s = '"'+self.name+'": {' + for e in self.dimensions: + s += e.get_json_field_string() + ',' + s = s[:-1] + '}' + return s + + + def get_json_field_string(self): + if random.random() < self.percent_nulls: + s = '"'+self.name+'": null' + else: + if self.cardinality is None: + s = self.get_instance() + else: + index = int(self.cardinality_distribution.get_sample()) + if index < 0: + index = 0 + if index >= len(self.cardinality): + index = len(self.cardinality)-1 + s = self.cardinality[index] + return s + + def is_missing(self): + return random.random() < self.percent_missing + +class ElementList(): + def __init__(self, desc): + self.name = desc['name'] + self.elements = get_variables(desc['elements']) + self.length_distribution = parse_distribution(desc['length_distribution']) + self.selection_distribution = parse_distribution(desc['selection_distribution']) + if 'percent_nulls' in desc.keys(): + self.percent_nulls = desc['percent_nulls'] / 100.0 + else: + self.percent_nulls = 0.0 + if 'percent_missing' in desc.keys(): + self.percent_missing = desc['percent_missing'] / 100.0 + else: + self.percent_missing = 0.0 + cardinality = desc['cardinality'] + if cardinality == 0: + self.cardinality = None + self.cardinality_distribution = None + else: + self.cardinality = [] + if 'cardinality_distribution' not in desc.keys(): + print('Element '+self.name+' specifies a cardinality without a cardinality distribution') + exit() + self.cardinality_distribution = parse_distribution(desc['cardinality_distribution']) + for i in range(cardinality): + value = None + while True: + value = self.get_instance() + if value not in self.cardinality: + break + self.cardinality.append(value) + + def __str__(self): + s = 'ElementObject(name='+self.name + s += ', length_distribution='+str(self.length_distribution) + s += ', selection_distribution='+str(self.selection_distribution) + s += ', elements=[' + for e in self.elements: + s += ',' + str(e) + s += '])' + return s + + def get_instance(self): + s = '"'+self.name+'": [' + length = int(self.length_distribution.get_sample()) + for i in range(length): + index = int(self.selection_distribution.get_sample()) + if index < 0: + index = 0 + if index >= length: + index = length-1 + s += re.sub('^.*?:', '', self.elements[index].get_json_field_string(), count=1) + ',' + s = s[:-1] + ']' + return s + + + def get_json_field_string(self): + if random.random() < self.percent_nulls: + s = '"'+self.name+'": null' + else: + if self.cardinality is None: + s = self.get_instance() + else: + index = int(self.cardinality_distribution.get_sample()) + if index < 0: + index = 0 + if index >= len(self.cardinality): + index = len(self.cardinality)-1 + s = self.cardinality[index] + return s + + def is_missing(self): + return random.random() < self.percent_missing + + +def parse_element(desc): + if desc['type'].lower() == 'counter': + el = ElementCounter(desc) + elif desc['type'].lower() == 'enum': + el = ElementEnum(desc) + elif desc['type'].lower() == 'string': + el = ElementString(desc) + elif desc['type'].lower() == 'int': + el = ElementInt(desc) + elif desc['type'].lower() == 'float': + el = ElementFloat(desc) + elif desc['type'].lower() == 'timestamp': + el = ElementTimestamp(desc) + elif desc['type'].lower() == 'ipaddress': + el = ElementIPAddress(desc) + elif desc['type'].lower() == 'variable': + el = ElementVariable(desc) + elif desc['type'].lower() == 'object': + el = ElementObject(desc) + elif desc['type'].lower() == 'list': + el = ElementList(desc) + else: + print('Error: Unknown dimension type "'+desc['type']+'"') + exit() + return el + + +def get_variables(desc): + elements = [] + for element in desc: + el = parse_element(element) + elements.append(el) + return elements + +def get_dimensions(desc, global_clock): + elements = get_variables(desc) + elements.insert(0, ElementNow(global_clock)) + return elements + + +# +# Set up the state machine +# + +class Transition: + def __init__(self, next_state, probability): + self.next_state = next_state + self.probability = probability + + def __str__(self): + return 'Transition(next_state='+str(self.next_state)+', probability='+str(self.probability)+')' + +def parse_transitions(desc): + transitions = [] + for trans in desc: + next_state = trans['next'] + probability = float(trans['probability']) + transitions.append(Transition(next_state, probability)) + return transitions + +class State: + def __init__(self, name, dimensions, delay, transitions, variables): + self.name = name + self.dimensions = dimensions + self.delay = delay + self.transistion_states = [t.next_state for t in transitions] + self.transistion_probabilities = [t.probability for t in transitions] + self.variables = variables + + def __str__(self): + return 'State(name='+self.name+', dimensions='+str([str(d) for d in self.dimensions])+', delay='+str(self.delay)+', transistion_states='+str(self.transistion_states)+', transistion_probabilities='+str(self.transistion_probabilities)+'variables='+str([str(v) for v in self.variables])+')' + + def get_next_state_name(self): + return random.choices(self.transistion_states, weights=self.transistion_probabilities, k=1)[0] + +class SimEnd: + lock = threading.Lock() + thread_end_event = threading.Event() + def __init__(self, total_recs, runtime, global_clock): + self.total_recs = total_recs + self.record_count = 0 + self.global_clock = global_clock + if runtime is None: + self.t = None + else: + if runtime[-1].lower() == 's': + self.t = int(runtime[:-1]) + elif runtime[-1].lower() == 'm': + self.t = int(runtime[:-1]) * 60 + elif runtime[-1].lower() == 'h': + self.t = int(runtime[:-1]) * 60 * 60 + else: + print('Error: Unknown runtime value"'+runtime+'"') + exit() + + def inc_rec_count(self): + self.lock.acquire() + self.record_count += 1 + self.lock.release() + if (self.total_recs is not None) and (self.record_count >= self.total_recs): + self.thread_end_event.set() + + def is_done(self): + return ((self.total_recs is not None) and (self.record_count >= self.total_recs)) or ((self.t is not None) and self.thread_end_event.is_set()) + + def wait_for_end(self): + if self.t is not None: + self.global_clock.activate_thread() + self.global_clock.sleep(self.t) + self.thread_end_event.set() + self.global_clock.deactivate_thread() + elif self.total_recs is not None: + self.thread_end_event.wait() + self.global_clock.release_all() + else: + while True: + time.sleep(60) + + +# +# Run the driver +# + +def create_record(dimensions, variables): + json_string = '{' + for element in dimensions: + if isinstance(element, ElementVariable): + json_string += element.get_json_field_string(variables) + ',' + else: + if isinstance(element, ElementNow) or not element.is_missing(): + json_string += element.get_json_field_string() + ',' + json_string = json_string[:-1] + '}' + return json_string + +def set_variable_values(variables, dimensions): + for d in dimensions: + variables[d.name] = d.get_stochastic_value() + +def worker_thread(target_printer, states, initial_state, sim_end, global_clock): + # Process the state machine using worker threads + #print('Thread '+threading.current_thread().name+' starting...') + global_clock.activate_thread() + current_state = initial_state + variables = {} + while True: + set_variable_values(variables, current_state.variables) + record = create_record(current_state.dimensions, variables) + target_printer.print(record) + sim_end.inc_rec_count() + if sim_end.is_done(): + break + delta = float(current_state.delay.get_sample()) + global_clock.sleep(delta) + if sim_end.is_done(): + break + next_state_name = current_state.get_next_state_name() + if next_state_name.lower() == 'stop': + break + current_state = states[next_state_name] + + #print('Thread '+threading.current_thread().name+' done!') + global_clock.end_thread() + +def spawning_thread(target_printer, rate_delay, states, initial_state, sim_end, global_clock): + #print('Thread '+threading.current_thread().name+' starting...') + global_clock.activate_thread() + # Spawn the workers in a separate thread so we can stop the whole thing in the middle of spawning if necessary + count = 0 + while not sim_end.is_done(): + thread_name = 'W'+str(count) + + count += 1 + t = threading.Thread(target=worker_thread, args=(target_printer, states, initial_state, sim_end, global_clock, ), name=thread_name, daemon=True) + t.start() + global_clock.sleep(float(rate_delay.get_sample())) + global_clock.end_thread() + #print('Thread '+threading.current_thread().name+' done!') + + +def simulate(config_file_name, runtime, total_recs, time_type, start_time): + + if config_file_name: + with open(config_file_name, 'r') as f: + config = json.load(f) + else: + config = json.load(sys.stdin) + + # + # Set up the gloabl clock + # + + global_clock = Clock(time_type, start_time) + sim_end = SimEnd(total_recs, runtime, global_clock) + + + # + # Set up the output target + # + + target = config['target'] + + if target['type'].lower() == 'stdout': + target_printer = PrintStdout() + elif target['type'].lower() == 'file': + path = target['path'] + if path is None: + print('Error: File target requires a path item') + exit() + target_printer = PrintFile(path) + elif target['type'].lower() == 'kafka': + if 'endpoint' in target.keys(): + endpoint = target['endpoint'] + else: + print('Error: Kafka target requires an endpoint item') + exit() + if 'topic' in target.keys(): + topic = target['topic'] + else: + print('Error: Kafka target requires a topic item') + exit() + if 'security_protocol' in target.keys(): + security_protocol = target['security_protocol'] + else: + security_protocol = 'PLAINTEXT' + if 'compression_type' in target.keys(): + compression_type = target['compression_type'] + else: + compression_type = None + target_printer = PrintKafka(endpoint, topic, security_protocol, compression_type) + elif target['type'].lower() == 'confluent': + if 'servers' in target.keys(): + servers = target['servers'] + else: + print('Error: Conlfuent target requires a servers item') + exit() + if 'topic' in target.keys(): + topic = target['topic'] + else: + print('Error: Confluent target requires a topic item') + exit() + if 'username' in target.keys(): + username = target['username'] + else: + print('Error: Confluent target requires a username') + exit() + if 'password' in target.keys(): + password = target['password'] + else: + print('Error: Confluent target requires a password') + exit() + target_printer = PrintConfluent(servers, topic, username, password) + else: + print('Error: Unknown target type "'+target['type']+'"') + exit() + + #sys.stderr.write('target='+str(target_printer)+'\n') + + + # + # Set up the interarrival rate + # + + rate = config['interarrival'] + rate_delay = parse_distribution(rate) + + #sys.stderr.write('rate_delay='+str(rate_delay)+'\n') + + + # + # Set up emitters list + # + + emitters = {} + for emitter in config['emitters']: + name = emitter['name'] + dimensions = get_dimensions(emitter['dimensions'], global_clock) + emitters[name] = dimensions + + #sys.stderr.write('emitters='+str(['(name='+str(key)+', dimensions='+str([str(e) for e in emitters[key]])+')' for key in emitters])+'\n') + + + # + # Set up the state machine + # + + state_desc = config['states'] + initial_state = None + states = {} + for state in state_desc: + name = state['name'] + emitter_name = state['emitter'] + if 'variables' not in state.keys(): + variables = [] + else: + variables = get_variables(state['variables']) + dimensions = emitters[emitter_name] + delay = parse_distribution(state['delay']) + transitions = parse_transitions(state['transitions']) + this_state = State(name, dimensions, delay, transitions, variables) + states[name] = this_state + if initial_state is None: + initial_state = this_state + + #sys.stderr.write('states='+str(['('+str(key)+':'+str(states[key])+')' for key in states])+'\n') + + # + # Finally, start the simulation + # + + thrd = threading.Thread(target=spawning_thread, args=(target_printer, rate_delay, states, initial_state, sim_end, global_clock, ), name='Spawning', daemon=True) + thrd.start() + sim_end.wait_for_end() + +def main(): + + # + # Parse the command line + # + + parser = argparse.ArgumentParser(description='Generates JSON records as a workload for Apache Druid.') + #parser.add_argument('config_file', metavar='', help='the workload config file name') + parser.add_argument('-f', dest='config_file', nargs='?', help='the workload config file name') + parser.add_argument('-t', dest='time', nargs='?', help='the script runtime (may not be used with -n)') + parser.add_argument('-n', dest='n_recs', nargs='?', help='the number of records to generate (may not be used with -t)') + parser.add_argument('-s', dest='time_type', nargs='?', const='SIM', default='REAL', help='simulate time (default is real, not simulated)') + + args = parser.parse_args() + + config_file_name = args.config_file + runtime = args.time + total_recs = None + if args.n_recs is not None: + total_recs = int(args.n_recs) + time_type = args.time_type + if time_type == 'SIM': + start_time = datetime.now() + elif time_type == 'REAL': + start_time = datetime.now() + else: + start_time = dateutil.parser.isoparse(time_type) + time_type = 'SIM' + + + if (runtime is not None) and (total_recs is not None): + print("Use either -t or -n, but not both") + parser.print_help() + exit() + + + simulate(config_file_name, runtime, total_recs, time_type, start_time) + + + +if __name__ == "__main__": + main() diff --git a/examples/quickstart/jupyter-notebooks/docker-jupyter/kafka_docker_config.json b/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/kafka_docker_config.json similarity index 100% rename from examples/quickstart/jupyter-notebooks/docker-jupyter/kafka_docker_config.json rename to examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/kafka_docker_config.json diff --git a/examples/quickstart/jupyter-notebooks/sql-tutorial.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/03-query/00-using-sql-with-druidapi.ipynb similarity index 99% rename from examples/quickstart/jupyter-notebooks/sql-tutorial.ipynb rename to examples/quickstart/jupyter-notebooks/notebooks/03-query/00-using-sql-with-druidapi.ipynb index b0ce6238e92..4d9349d8f2b 100644 --- a/examples/quickstart/jupyter-notebooks/sql-tutorial.ipynb +++ b/examples/quickstart/jupyter-notebooks/notebooks/03-query/00-using-sql-with-druidapi.ipynb @@ -661,7 +661,7 @@ "name": "python", "nbconvert_exporter": "python", "pygments_lexer": "ipython3", - "version": "3.8.16" + "version": "3.11.4" }, "toc-autonumbering": false, "toc-showcode": false, diff --git a/examples/quickstart/jupyter-notebooks/api-tutorial.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/04-api/00-getting-started.ipynb similarity index 99% rename from examples/quickstart/jupyter-notebooks/api-tutorial.ipynb rename to examples/quickstart/jupyter-notebooks/notebooks/04-api/00-getting-started.ipynb index b2616b5d8e2..f2e920a51fd 100644 --- a/examples/quickstart/jupyter-notebooks/api-tutorial.ipynb +++ b/examples/quickstart/jupyter-notebooks/notebooks/04-api/00-getting-started.ipynb @@ -681,7 +681,7 @@ "name": "python", "nbconvert_exporter": "python", "pygments_lexer": "ipython3", - "version": "3.8.16" + "version": "3.11.4" }, "vscode": { "interpreter": { From f8f2fe8b7b06b7b2f08c5d3ae6c99de0a07bcf31 Mon Sep 17 00:00:00 2001 From: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com> Date: Thu, 22 Jun 2023 12:27:23 +0530 Subject: [PATCH 13/74] Skip tests based on files changed in the PR (#14445) Our CI system has a lot of tests. And much of this testing is really unnecessary for most of the PRs. This PR adds some checks so we can skip these expensive tests when we know they are not necessary. --- .github/workflows/revised-its.yml | 26 ++++++++++++ .github/workflows/standard-its.yml | 37 +++++++++++++++- .github/workflows/unit-tests.yml | 42 +++++++++++++++++-- .../Common/environment-configs/common.env | 1 + .../docker/environment-configs/common | 1 + .../docker/environment-configs/common-ldap | 1 + .../test-groups/custom-coordinator-duties | 1 + .../test-groups/prepopulated-data | 1 + .../test-groups/shuffle-deep-store | 1 + 9 files changed, 106 insertions(+), 5 deletions(-) diff --git a/.github/workflows/revised-its.yml b/.github/workflows/revised-its.yml index e10238b5e16..900b308569d 100644 --- a/.github/workflows/revised-its.yml +++ b/.github/workflows/revised-its.yml @@ -21,7 +21,30 @@ on: workflow_dispatch: jobs: + changes: + runs-on: ubuntu-latest + # Required permissions + permissions: + pull-requests: read + # Set job outputs to values from filter step + outputs: + # run everything if not a PR + core: ${{ steps.filter.outputs.core || github.event_name != 'pull_request'}} + # the common extension in revised ITs is different from the one in standard ITs + common-extensions: ${{ steps.filter.outputs.common-extensions }} + steps: + - uses: dorny/paths-filter@v2 + if: github.event_name == 'pull_request' + id: filter + with: + filters: | + common-extensions: + - 'extension-core/(mysql-metadata-storage|druid-it-tools|druid-lookups-cached-global|druid-histogram|druid-datasketches|druid-parquet-extensions|druid-avro-extensions|druid-protobuf-extensions|druid-orc-extensions|druid-kafka-indexing-service|druid-s3-extensions|druid-multi-stage-query|druid-catalog)/**' + core: + - '!extension*/**' + it: + needs: changes strategy: fail-fast: false matrix: @@ -31,6 +54,7 @@ jobs: #indexer: [indexer, middleManager] indexer: [middleManager] uses: ./.github/workflows/reusable-revised-its.yml + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: build_jdk: ${{ matrix.jdk }} runtime_jdk: ${{ matrix.jdk }} @@ -40,7 +64,9 @@ jobs: mysql_driver: com.mysql.jdbc.Driver s3-deep-storage-minio: + needs: changes uses: ./.github/workflows/reusable-revised-its.yml + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: build_jdk: 8 runtime_jdk: 11 diff --git a/.github/workflows/standard-its.yml b/.github/workflows/standard-its.yml index 155105fab3d..2f17222ada0 100644 --- a/.github/workflows/standard-its.yml +++ b/.github/workflows/standard-its.yml @@ -21,12 +21,35 @@ on: workflow_dispatch: jobs: + changes: + runs-on: ubuntu-latest + # Required permissions + permissions: + pull-requests: read + # Set job outputs to values from filter step + outputs: + # run everything if not a PR + core: ${{ steps.filter.outputs.core || github.event_name != 'pull_request'}} + common-extensions: ${{ steps.filter.outputs.common-extensions }} + steps: + - uses: dorny/paths-filter@v2 + if: github.event_name == 'pull_request' + id: filter + with: + filters: | + common-extensions: + - 'extension-core/(mysql-metadata-storage|druid-basic-security|simple-client-sslcontext|druid-testing-tools|druid-lookups-cached-global|druid-histogram|druid-datasketches|druid-parquet-extensions|druid-avro-extensions|druid-protobuf-extensions|druid-orc-extensions|druid-kafka-indexing-service|druid-s3-extensions)/**' + core: + - '!extension*/**' + integration-index-tests-middleManager: + needs: changes strategy: fail-fast: false matrix: testing_group: [batch-index, input-format, input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction] uses: ./.github/workflows/reusable-standard-its.yml + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: build_jdk: 8 runtime_jdk: 8 @@ -35,11 +58,13 @@ jobs: group: ${{ matrix.testing_group }} integration-index-tests-indexer: + needs: changes strategy: fail-fast: false matrix: - testing_group: [input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-transactional-index, kafka-index-slow, kafka-transactional-index-slow, kafka-data-format, append-ingestion, compaction] + testing_group: [input-source, perfect-rollup-parallel-batch-index, kafka-index, append-ingestion, compaction] uses: ./.github/workflows/reusable-standard-its.yml + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: build_jdk: 8 runtime_jdk: 8 @@ -48,11 +73,13 @@ jobs: group: ${{ matrix.testing_group }} integration-query-tests-middleManager: + needs: changes strategy: fail-fast: false matrix: testing_group: [query, query-retry, query-error, security, high-availability] uses: ./.github/workflows/reusable-standard-its.yml + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: build_jdk: 8 runtime_jdk: 8 @@ -62,11 +89,13 @@ jobs: group: ${{ matrix.testing_group }} integration-query-tests-middleManager-mariaDB: + needs: changes strategy: fail-fast: false matrix: jdk: [8, 11] uses: ./.github/workflows/reusable-standard-its.yml + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: build_jdk: 8 runtime_jdk: ${{ matrix.jdk }} @@ -77,11 +106,13 @@ jobs: group: query integration-shuffle-deep-store-tests: + needs: changes strategy: fail-fast: false matrix: indexer: [indexer, middleManager] uses: ./.github/workflows/reusable-standard-its.yml + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: build_jdk: 8 runtime_jdk: 8 @@ -91,7 +122,9 @@ jobs: group: shuffle deep store integration-custom-coordinator-duties-tests: + needs: changes uses: ./.github/workflows/reusable-standard-its.yml + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: build_jdk: 8 runtime_jdk: 8 @@ -101,7 +134,9 @@ jobs: group: custom coordinator duties integration-k8s-leadership-tests: + needs: changes name: (Compile=openjdk8, Run=openjdk8, Cluster Build On K8s) ITNestedQueryPushDownTest integration test + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} runs-on: ubuntu-22.04 env: MVN: mvn --no-snapshot-updates diff --git a/.github/workflows/unit-tests.yml b/.github/workflows/unit-tests.yml index f305ba9887a..1f75cc4eed8 100644 --- a/.github/workflows/unit-tests.yml +++ b/.github/workflows/unit-tests.yml @@ -31,14 +31,44 @@ on: description: 'Flag to decide if next tests need to run incase coverage issue failures' value: | ${{ - (jobs.indexing_modules_test.result == 'success' || fromJson(jobs.indexing_modules_test.outputs.coverage_failure)) && - (jobs.processing_modules_test.result == 'success' || fromJson(jobs.processing_modules_test.outputs.coverage_failure)) && - (jobs.server_modules_test.result == 'success' || fromJson(jobs.server_modules_test.outputs.coverage_failure)) && - (jobs.other_modules_test.result == 'success' || fromJson(jobs.other_modules_test.outputs.coverage_failure)) + (jobs.indexing_modules_test.result == 'success' || jobs.indexing_modules_test.result == 'skipped' || + fromJson(jobs.indexing_modules_test.outputs.coverage_failure)) && + (jobs.processing_modules_test.result == 'success' || jobs.processing_modules_test.result == 'skipped' || + fromJson(jobs.processing_modules_test.outputs.coverage_failure)) && + (jobs.server_modules_test.result == 'success' || jobs.server_modules_test.result == 'skipped' || fromJson + (jobs.server_modules_test.outputs.coverage_failure)) && + (jobs.other_modules_test.result == 'success' || jobs.other_modules_test.result == 'skipped' || fromJson(jobs + .other_modules_test.outputs.coverage_failure)) }} jobs: + changes: + runs-on: ubuntu-latest + # Required permissions + permissions: + pull-requests: read + # Set job outputs to values from filter step + outputs: + kafka: ${{ steps.filter.outputs.kafka }} + kinesis: ${{ steps.filter.outputs.kinesis }} + # run everything if not a PR + core: ${{ steps.filter.outputs.core || github.event_name != 'pull_request'}} + steps: + - uses: dorny/paths-filter@v2 + if: github.event_name == 'pull_request' + id: filter + with: + filters: | + core: + - '!extension*/**' + kafka: + - 'extensions-core/kafka-indexing-service/**' + kinesis: + - 'extensions-core/kinesis-indexing-service/**' + indexing_modules_test: + needs: changes + if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.kafka == 'true' || needs.changes.outputs.kinesis == 'true'}} uses: ./.github/workflows/reusable-unit-tests.yml with: jdk: ${{ inputs.jdk }} @@ -47,6 +77,8 @@ jobs: maven_projects: 'indexing-hadoop,indexing-service,extensions-core/kafka-indexing-service,extensions-core/kinesis-indexing-service' processing_modules_test: + needs: changes + if: ${{ needs.changes.outputs.core == 'true' }} uses: ./.github/workflows/reusable-unit-tests.yml with: jdk: ${{ inputs.jdk }} @@ -55,6 +87,8 @@ jobs: maven_projects: 'processing' server_modules_test: + needs: changes + if: ${{ needs.changes.outputs.core == 'true' }} uses: ./.github/workflows/reusable-unit-tests.yml with: jdk: ${{ inputs.jdk }} diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/common.env b/integration-tests-ex/cases/cluster/Common/environment-configs/common.env index 8b58cd949e3..6f9dd9ba94f 100644 --- a/integration-tests-ex/cases/cluster/Common/environment-configs/common.env +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/common.env @@ -49,6 +49,7 @@ DRUID_INSTANCE= # variables: druid_standard_loadList defined here, and druid_test_loadList, defined # in a docker-compose.yaml file, for any test-specific extensions. # See compose.md for more details. +# If you are making a change in load list below, make the necessary changes in github actions too druid_standard_loadList=mysql-metadata-storage,druid-it-tools,druid-lookups-cached-global,druid-histogram,druid-datasketches,druid-parquet-extensions,druid-avro-extensions,druid-protobuf-extensions,druid-orc-extensions,druid-kafka-indexing-service,druid-s3-extensions,druid-multi-stage-query,druid-catalog # Location of Hadoop dependencies provided at runtime in the shared directory. diff --git a/integration-tests/docker/environment-configs/common b/integration-tests/docker/environment-configs/common index a1a294bb1bf..1aec119e8d4 100644 --- a/integration-tests/docker/environment-configs/common +++ b/integration-tests/docker/environment-configs/common @@ -26,6 +26,7 @@ COMMON_DRUID_JAVA_OPTS=-Duser.timezone=UTC -Dfile.encoding=UTF-8 -Dlog4j.configu DRUID_DEP_LIB_DIR=/shared/hadoop_xml:/shared/docker/lib/*:/usr/local/druid/lib/mysql-connector-java.jar # Druid configs +# If you are making a change in load list below, make the necessary changes in github actions too druid_extensions_loadList=["mysql-metadata-storage","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches","druid-parquet-extensions","druid-avro-extensions","druid-protobuf-extensions","druid-orc-extensions","druid-kafka-indexing-service","druid-s3-extensions"] druid_startup_logging_logProperties=true druid_extensions_directory=/shared/docker/extensions diff --git a/integration-tests/docker/environment-configs/common-ldap b/integration-tests/docker/environment-configs/common-ldap index b49aa1a45cb..261752d8bad 100644 --- a/integration-tests/docker/environment-configs/common-ldap +++ b/integration-tests/docker/environment-configs/common-ldap @@ -27,6 +27,7 @@ COMMON_DRUID_JAVA_OPTS=-Duser.timezone=UTC -Dfile.encoding=UTF-8 -Dlog4j.configu DRUID_DEP_LIB_DIR=/shared/hadoop_xml:/shared/docker/lib/*:/usr/local/druid/lib/mysql-connector-java.jar # Druid configs +# If you are making a change in load list below, make the necessary changes in github actions too druid_extensions_loadList=["mysql-metadata-storage","druid-s3-extensions","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches"] druid_extensions_directory=/shared/docker/extensions druid_auth_authenticator_ldap_authorizerName=ldapauth diff --git a/integration-tests/docker/environment-configs/test-groups/custom-coordinator-duties b/integration-tests/docker/environment-configs/test-groups/custom-coordinator-duties index 0fd17a49617..5c75c4197d3 100644 --- a/integration-tests/docker/environment-configs/test-groups/custom-coordinator-duties +++ b/integration-tests/docker/environment-configs/test-groups/custom-coordinator-duties @@ -17,6 +17,7 @@ # under the License. # +# If you are making a change in load list below, make the necessary changes in github actions too druid_extensions_loadList=["druid-kafka-indexing-service","mysql-metadata-storage","druid-s3-extensions","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches"] druid_coordinator_period_metadataStoreManagementPeriod=PT1H diff --git a/integration-tests/docker/environment-configs/test-groups/prepopulated-data b/integration-tests/docker/environment-configs/test-groups/prepopulated-data index acce51db6d6..deb59dfe402 100644 --- a/integration-tests/docker/environment-configs/test-groups/prepopulated-data +++ b/integration-tests/docker/environment-configs/test-groups/prepopulated-data @@ -19,6 +19,7 @@ AWS_REGION=us-east-1 +# If you are making a change in load list below, make the necessary changes in github actions too druid_extensions_loadList=["mysql-metadata-storage","druid-s3-extensions","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches","druid-integration-tests"] # Setting s3 credentials and region to use pre-populated data for testing. diff --git a/integration-tests/docker/environment-configs/test-groups/shuffle-deep-store b/integration-tests/docker/environment-configs/test-groups/shuffle-deep-store index 70a6a65940c..731d7254f85 100644 --- a/integration-tests/docker/environment-configs/test-groups/shuffle-deep-store +++ b/integration-tests/docker/environment-configs/test-groups/shuffle-deep-store @@ -19,5 +19,6 @@ # Test with deep storage as intermediate location to store shuffle data # Local deep storage will be used here +# If you are making a change in load list below, make the necessary changes in github actions too druid_extensions_loadList=["mysql-metadata-storage","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches"] druid_processing_intermediaryData_storage_type=deepstore From 90b8f850a5e948776eaa2e3b1ba56206b96e1e57 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 22 Jun 2023 14:44:06 +0530 Subject: [PATCH 14/74] Allow empty tiered replicants map for load rules (#14432) Changes: - Add property `useDefaultTierForNull` for all load rules. This property determines the default value of `tieredReplicants` if it is not specified. When true, the default is `_default_tier => 2 replicas`. When false, the default is empty, i.e. no replicas on any tier. - Fix validation to allow empty replicants map, so that the segment is used but not loaded anywhere. --- docs/operations/rule-configuration.md | 14 +- .../metadata/SQLMetadataRuleManager.java | 3 +- .../coordinator/rules/ForeverLoadRule.java | 46 +----- .../coordinator/rules/IntervalLoadRule.java | 43 ++--- .../server/coordinator/rules/LoadRule.java | 83 +++++++++- .../coordinator/rules/PeriodLoadRule.java | 47 +++--- .../metadata/SQLMetadataRuleManagerTest.java | 22 ++- .../coordinator/BalanceSegmentsProfiler.java | 2 +- .../coordinator/DruidCoordinatorTest.java | 6 +- .../server/coordinator/duty/RunRulesTest.java | 72 ++++++--- .../duty/UnloadUnusedSegmentsTest.java | 6 +- .../rules/ForeverLoadRuleTest.java | 100 ++++++++---- .../rules/IntervalLoadRuleTest.java | 104 +++++++++--- .../coordinator/rules/LoadRuleTest.java | 12 +- .../coordinator/rules/PeriodLoadRuleTest.java | 151 ++++++++++++++---- .../CoordinatorSimulationBaseTest.java | 2 +- .../simulate/TestMetadataRuleManager.java | 2 +- .../server/http/DataSourcesResourceTest.java | 2 +- .../router/CoordinatorRuleManagerTest.java | 12 +- .../router/TieredBrokerHostSelectorTest.java | 7 +- 20 files changed, 497 insertions(+), 239 deletions(-) diff --git a/docs/operations/rule-configuration.md b/docs/operations/rule-configuration.md index 0d75cf54e89..0bf803355aa 100644 --- a/docs/operations/rule-configuration.md +++ b/docs/operations/rule-configuration.md @@ -109,7 +109,16 @@ In the web console you can use the up and down arrows on the right side of the i Load rules define how Druid assigns segments to [historical process tiers](./mixed-workloads.md#historical-tiering), and how many replicas of a segment exist in each tier. -If you have a single tier, Druid automatically names the tier `_default` and loads all segments onto it. If you define an additional tier, you must define a load rule to specify which segments to load on that tier. Until you define a load rule, your new tier remains empty. +If you have a single tier, Druid automatically names the tier `_default`. If you define an additional tier, you must define a load rule to specify which segments to load on that tier. Until you define a load rule, your new tier remains empty. + +All load rules can have these properties: + +|Property|Description|Required|Default value| +|---------|-----------|---------|-------------| +| `tieredReplicants`| Map from tier names to the respective number of segment replicas to be loaded on those tiers. The number of replicas for each tier must be either 0 or a positive integer.| No | When `useDefaultTierForNull` is `true`, the default value is `{"_default_tier": 2}` i.e. 2 replicas to be loaded on the `_default_tier`.

When `useDefaultTierForNull` is `false`, the default value is `{}` i.e. no replicas to be loaded on any tier. | +|`useDefaultTierForNull`|Determines the default value of `tieredReplicants` if it is not specified or set to `null`.| No | `true`| + +Specific types of load rules discussed below may have other properties too. ### Forever load rule @@ -130,6 +139,7 @@ The following example places one replica of each segment on a custom tier named Set the following property: - `tieredReplicants`: a map of tier names to the number of segment replicas for that tier. +- `useDefaultTierForNull`: This parameter determines the default value of `tieredReplicants` and only has an effect if the field is not present. The default value of `useDefaultTierForNull` is true. ### Period load rule @@ -158,6 +168,7 @@ Set the following properties: You can use this property to load segments with future start and end dates, where "future" is relative to the time when the Coordinator evaluates data against the rule. Defaults to `true`. - `tieredReplicants`: a map of tier names to the number of segment replicas for that tier. +- `useDefaultTierForNull`: This parameter determines the default value of `tieredReplicants` and only has an effect if the field is not present. The default value of `useDefaultTierForNull` is true. ### Interval load rule @@ -180,6 +191,7 @@ Set the following properties: - `interval`: the load interval specified as an [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) range encoded as a string. - `tieredReplicants`: a map of tier names to the number of segment replicas for that tier. +- `useDefaultTierForNull`: This parameter determines the default value of `tieredReplicants` and only has an effect if the field is not present. The default value of `useDefaultTierForNull` is true. ## Drop rules diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java index b3d0fa9b27f..b7cdb2f7ec9 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java @@ -96,7 +96,8 @@ public class SQLMetadataRuleManager implements MetadataRuleManager ImmutableMap.of( DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS - ) + ), + null ) ); final String version = DateTimes.nowUtc().toString(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverLoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverLoadRule.java index dc1798f13e0..35f22fa555f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverLoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverLoadRule.java @@ -21,30 +21,24 @@ package org.apache.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.client.DruidServer; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Map; -import java.util.Objects; /** */ public class ForeverLoadRule extends LoadRule { - private final Map tieredReplicants; - @JsonCreator public ForeverLoadRule( - @JsonProperty("tieredReplicants") Map tieredReplicants + @JsonProperty("tieredReplicants") Map tieredReplicants, + @JsonProperty("useDefaultTierForNull") @Nullable Boolean useDefaultTierForNull ) { - this.tieredReplicants = tieredReplicants == null - ? ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) - : tieredReplicants; - validateTieredReplicants(this.tieredReplicants); + super(tieredReplicants, useDefaultTierForNull); } @Override @@ -54,20 +48,6 @@ public class ForeverLoadRule extends LoadRule return "loadForever"; } - @Override - @JsonProperty - public Map getTieredReplicants() - { - return tieredReplicants; - } - - @Override - public int getNumReplicants(String tier) - { - Integer retVal = tieredReplicants.get(tier); - return (retVal == null) ? 0 : retVal; - } - @Override public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { @@ -80,22 +60,4 @@ public class ForeverLoadRule extends LoadRule return true; } - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ForeverLoadRule that = (ForeverLoadRule) o; - return Objects.equals(tieredReplicants, that.tieredReplicants); - } - - @Override - public int hashCode() - { - return Objects.hash(tieredReplicants); - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalLoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalLoadRule.java index 2e944bf2854..209f5c24d1e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalLoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalLoadRule.java @@ -21,14 +21,14 @@ package org.apache.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Map; +import java.util.Objects; /** */ @@ -37,16 +37,15 @@ public class IntervalLoadRule extends LoadRule private static final Logger log = new Logger(IntervalLoadRule.class); private final Interval interval; - private final Map tieredReplicants; @JsonCreator public IntervalLoadRule( @JsonProperty("interval") Interval interval, - @JsonProperty("tieredReplicants") Map tieredReplicants + @JsonProperty("tieredReplicants") Map tieredReplicants, + @JsonProperty("useDefaultTierForNull") @Nullable Boolean useDefaultTierForNull ) { - this.tieredReplicants = tieredReplicants == null ? ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) : tieredReplicants; - validateTieredReplicants(this.tieredReplicants); + super(tieredReplicants, useDefaultTierForNull); this.interval = interval; } @@ -57,20 +56,6 @@ public class IntervalLoadRule extends LoadRule return "loadByInterval"; } - @Override - @JsonProperty - public Map getTieredReplicants() - { - return tieredReplicants; - } - - @Override - public int getNumReplicants(String tier) - { - final Integer retVal = tieredReplicants.get(tier); - return retVal == null ? 0 : retVal; - } - @JsonProperty public Interval getInterval() { @@ -98,24 +83,16 @@ public class IntervalLoadRule extends LoadRule if (o == null || getClass() != o.getClass()) { return false; } - + if (!super.equals(o)) { + return false; + } IntervalLoadRule that = (IntervalLoadRule) o; - - if (interval != null ? !interval.equals(that.interval) : that.interval != null) { - return false; - } - if (tieredReplicants != null ? !tieredReplicants.equals(that.tieredReplicants) : that.tieredReplicants != null) { - return false; - } - - return true; + return Objects.equals(interval, that.interval); } @Override public int hashCode() { - int result = interval != null ? interval.hashCode() : 0; - result = 31 * result + (tieredReplicants != null ? tieredReplicants.hashCode() : 0); - return result; + return Objects.hash(super.hashCode(), interval); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java index 548c25cad54..5d7b724c845 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java @@ -19,39 +19,106 @@ package org.apache.druid.server.coordinator.rules; -import org.apache.druid.java.util.common.IAE; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.client.DruidServer; +import org.apache.druid.common.config.Configs; +import org.apache.druid.error.InvalidInput; import org.apache.druid.timeline.DataSegment; import java.util.Map; +import java.util.Objects; /** * LoadRules indicate the number of replicants a segment should have in a given tier. */ public abstract class LoadRule implements Rule { + private final Map tieredReplicants; + /** + * Used to determing the default value if tieredReplicants is null in {@link #handleNullTieredReplicants}. + */ + private final boolean useDefaultTierForNull; + + protected LoadRule(Map tieredReplicants, Boolean useDefaultTierForNull) + { + this.useDefaultTierForNull = Configs.valueOrDefault(useDefaultTierForNull, true); + this.tieredReplicants = handleNullTieredReplicants(tieredReplicants, this.useDefaultTierForNull); + validateTieredReplicants(this.tieredReplicants); + } + + @JsonProperty + public Map getTieredReplicants() + { + return tieredReplicants; + } + + @JsonProperty + public boolean useDefaultTierForNull() + { + return useDefaultTierForNull; + } + @Override public void run(DataSegment segment, SegmentActionHandler handler) { handler.replicateSegment(segment, getTieredReplicants()); } - protected static void validateTieredReplicants(final Map tieredReplicants) + /** + * Returns the given {@code tieredReplicants} map unchanged if it is non-null (including empty). + * Returns the following default values if the given map is null. + *
    + *
  • If {@code useDefaultTierForNull} is true, returns a singleton map from {@link DruidServer#DEFAULT_TIER} to {@link DruidServer#DEFAULT_NUM_REPLICANTS}.
  • + *
  • If {@code useDefaultTierForNull} is false, returns an empty map. This causes segments to have a replication factor of 0 and not get assigned to any historical.
  • + *
+ */ + private static Map handleNullTieredReplicants(final Map tieredReplicants, boolean useDefaultTierForNull) { - if (tieredReplicants.size() == 0) { - throw new IAE("A rule with empty tiered replicants is invalid"); + if (useDefaultTierForNull) { + return Configs.valueOrDefault(tieredReplicants, ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS)); + } else { + return Configs.valueOrDefault(tieredReplicants, ImmutableMap.of()); } + } + + private static void validateTieredReplicants(final Map tieredReplicants) + { for (Map.Entry entry : tieredReplicants.entrySet()) { if (entry.getValue() == null) { - throw new IAE("Replicant value cannot be empty"); + throw InvalidInput.exception("Invalid number of replicas for tier [%s]. Value must not be null.", entry.getKey()); } if (entry.getValue() < 0) { - throw new IAE("Replicant value [%d] is less than 0, which is not allowed", entry.getValue()); + throw InvalidInput.exception("Invalid number of replicas for tier [%s]. Value [%d] must be positive.", entry.getKey(), entry.getValue()); } } } - public abstract Map getTieredReplicants(); + public int getNumReplicants(String tier) + { + Integer retVal = getTieredReplicants().get(tier); + return (retVal == null) ? 0 : retVal; + } - public abstract int getNumReplicants(String tier); + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LoadRule loadRule = (LoadRule) o; + return useDefaultTierForNull == loadRule.useDefaultTierForNull && Objects.equals( + tieredReplicants, + loadRule.tieredReplicants + ); + } + @Override + public int hashCode() + { + return Objects.hash(tieredReplicants, useDefaultTierForNull); + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodLoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodLoadRule.java index 0d6e2e099a7..1d2b4e18771 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodLoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodLoadRule.java @@ -21,15 +21,15 @@ package org.apache.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; +import javax.annotation.Nullable; import java.util.Map; +import java.util.Objects; /** */ @@ -40,17 +40,16 @@ public class PeriodLoadRule extends LoadRule private final Period period; private final boolean includeFuture; - private final Map tieredReplicants; @JsonCreator public PeriodLoadRule( @JsonProperty("period") Period period, @JsonProperty("includeFuture") Boolean includeFuture, - @JsonProperty("tieredReplicants") Map tieredReplicants + @JsonProperty("tieredReplicants") Map tieredReplicants, + @JsonProperty("useDefaultTierForNull") @Nullable Boolean useDefaultTierForNull ) { - this.tieredReplicants = tieredReplicants == null ? ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) : tieredReplicants; - validateTieredReplicants(this.tieredReplicants); + super(tieredReplicants, useDefaultTierForNull); this.period = period; this.includeFuture = includeFuture == null ? DEFAULT_INCLUDE_FUTURE : includeFuture; } @@ -74,20 +73,6 @@ public class PeriodLoadRule extends LoadRule return includeFuture; } - @Override - @JsonProperty - public Map getTieredReplicants() - { - return tieredReplicants; - } - - @Override - public int getNumReplicants(String tier) - { - final Integer retVal = tieredReplicants.get(tier); - return retVal == null ? 0 : retVal; - } - @Override public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { @@ -99,4 +84,26 @@ public class PeriodLoadRule extends LoadRule { return Rules.eligibleForLoad(period, interval, referenceTimestamp, includeFuture); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + PeriodLoadRule that = (PeriodLoadRule) o; + return includeFuture == that.includeFuture && Objects.equals(period, that.period); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), period, includeFuture); + } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java index 61bc5d9080d..20ffdb81188 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java @@ -108,7 +108,8 @@ public class SQLMetadataRuleManagerTest List rules = Collections.singletonList( new IntervalLoadRule( Intervals.of("2015-01-01/2015-02-01"), - ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) + ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), + null ) ); ruleManager.overrideRule(DATASOURCE, rules, createAuditInfo("override rule")); @@ -171,7 +172,8 @@ public class SQLMetadataRuleManagerTest List rules = Collections.singletonList( new IntervalLoadRule( Intervals.of("2015-01-01/2015-02-01"), - ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) + ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), + null ) ); final AuditInfo auditInfo = createAuditInfo("create audit entry"); @@ -200,9 +202,10 @@ public class SQLMetadataRuleManagerTest List rules = Collections.singletonList( new IntervalLoadRule( Intervals.of("2015-01-01/2015-02-01"), ImmutableMap.of( - DruidServer.DEFAULT_TIER, - DruidServer.DEFAULT_NUM_REPLICANTS - ) + DruidServer.DEFAULT_TIER, + DruidServer.DEFAULT_NUM_REPLICANTS + ), + null ) ); final AuditInfo auditInfo = createAuditInfo("test_comment"); @@ -232,7 +235,8 @@ public class SQLMetadataRuleManagerTest List rules = ImmutableList.of( new IntervalLoadRule( Intervals.of("2015-01-01/2015-02-01"), - ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) + ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), + null ) ); ruleManager.overrideRule(DATASOURCE, rules, createAuditInfo("test")); @@ -258,7 +262,8 @@ public class SQLMetadataRuleManagerTest List rules = ImmutableList.of( new IntervalLoadRule( Intervals.of("2015-01-01/2015-02-01"), - ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) + ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), + null ) ); ruleManager.overrideRule(DATASOURCE, rules, createAuditInfo("update rules")); @@ -286,7 +291,8 @@ public class SQLMetadataRuleManagerTest List rules = ImmutableList.of( new IntervalLoadRule( Intervals.of("2015-01-01/2015-02-01"), - ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) + ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), + null ) ); ruleManager.overrideRule(DATASOURCE, rules, createAuditInfo("update rules")); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java index b117b96520f..4a45e16bf96 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java @@ -59,7 +59,7 @@ public class BalanceSegmentsProfiler List segments = new ArrayList<>(); ServiceEmitter emitter; MetadataRuleManager manager; - PeriodLoadRule loadRule = new PeriodLoadRule(new Period("P5000Y"), null, ImmutableMap.of("normal", 3)); + PeriodLoadRule loadRule = new PeriodLoadRule(new Period("P5000Y"), null, ImmutableMap.of("normal", 3), null); List rules = ImmutableList.of(loadRule); @Before diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 6c0ab813b37..15fc5f5ac91 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -207,7 +207,7 @@ public class DruidCoordinatorTest extends CuratorTestBase String tier = "hot"; // Setup MetadataRuleManager - Rule foreverLoadRule = new ForeverLoadRule(ImmutableMap.of(tier, 2)); + Rule foreverLoadRule = new ForeverLoadRule(ImmutableMap.of(tier, 2), null); EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString())) .andReturn(ImmutableList.of(foreverLoadRule)).atLeastOnce(); @@ -325,8 +325,8 @@ public class DruidCoordinatorTest extends CuratorTestBase public void testCoordinatorTieredRun() throws Exception { final String dataSource = "dataSource", hotTierName = "hot", coldTierName = "cold"; - final Rule hotTier = new IntervalLoadRule(Intervals.of("2018-01-01/P1M"), ImmutableMap.of(hotTierName, 1)); - final Rule coldTier = new ForeverLoadRule(ImmutableMap.of(coldTierName, 1)); + final Rule hotTier = new IntervalLoadRule(Intervals.of("2018-01-01/P1M"), ImmutableMap.of(hotTierName, 1), null); + final Rule coldTier = new ForeverLoadRule(ImmutableMap.of(coldTierName, 1), null); final String loadPathCold = "/druid/loadqueue/cold:1234"; final DruidServer hotServer = new DruidServer("hot", "hot", null, 5L, ServerType.HISTORICAL, hotTierName, 0); final DruidServer coldServer = new DruidServer("cold", "cold", null, 5L, ServerType.HISTORICAL, coldTierName, 0); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java index abc218d8946..cd9c1e228da 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java @@ -129,7 +129,8 @@ public class RunRulesTest Collections.singletonList( new IntervalLoadRule( Intervals.of("2012-01-01/2012-01-02"), - ImmutableMap.of("normal", 2) + ImmutableMap.of("normal", 2), + null ) )).atLeastOnce(); EasyMock.replay(databaseRuleManager); @@ -185,7 +186,8 @@ public class RunRulesTest Collections.singletonList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("hot", 2, "normal", 2) + ImmutableMap.of("hot", 2, "normal", 2), + null ) )).atLeastOnce(); EasyMock.replay(databaseRuleManager); @@ -248,15 +250,18 @@ public class RunRulesTest Lists.newArrayList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T06:00:00.000Z"), - ImmutableMap.of("hot", 1) + ImmutableMap.of("hot", 1), + null ), new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("normal", 1) + ImmutableMap.of("normal", 1), + null ), new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("cold", 1) + ImmutableMap.of("cold", 1), + null ) )).atLeastOnce(); EasyMock.replay(databaseRuleManager); @@ -353,11 +358,13 @@ public class RunRulesTest Lists.newArrayList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T06:00:00.000Z"), - ImmutableMap.of("hot", 2) + ImmutableMap.of("hot", 2), + null ), new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("cold", 1) + ImmutableMap.of("cold", 1), + null ) ) ).atLeastOnce(); @@ -399,11 +406,13 @@ public class RunRulesTest Lists.newArrayList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("hot", 1) + ImmutableMap.of("hot", 1), + null ), new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("normal", 1) + ImmutableMap.of("normal", 1), + null ) ) ).atLeastOnce(); @@ -445,11 +454,13 @@ public class RunRulesTest Lists.newArrayList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("hot", 1) + ImmutableMap.of("hot", 1), + null ), new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z"), - ImmutableMap.of("normal", 1) + ImmutableMap.of("normal", 1), + null ) ) ).atLeastOnce(); @@ -480,7 +491,8 @@ public class RunRulesTest Collections.singletonList( new IntervalLoadRule( Intervals.of("2012-01-02T00:00:00.000Z/2012-01-03T00:00:00.000Z"), - ImmutableMap.of("normal", 1) + ImmutableMap.of("normal", 1), + null ) ) ) @@ -527,7 +539,8 @@ public class RunRulesTest Lists.newArrayList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("normal", 1) + ImmutableMap.of("normal", 1), + null ), new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) ) @@ -564,7 +577,8 @@ public class RunRulesTest Lists.newArrayList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("normal", 1) + ImmutableMap.of("normal", 1), + null ), new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) ) @@ -620,7 +634,8 @@ public class RunRulesTest Lists.newArrayList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("hot", 1) + ImmutableMap.of("hot", 1), + null ), new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) ) @@ -663,7 +678,8 @@ public class RunRulesTest Lists.newArrayList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T12:00:00.000Z"), - ImmutableMap.of("hot", 1) + ImmutableMap.of("hot", 1), + null ), new IntervalDropRule(Intervals.of("2012-01-01T00:00:00.000Z/2012-01-02T00:00:00.000Z")) ) @@ -703,7 +719,8 @@ public class RunRulesTest Collections.singletonList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2012-01-01T01:00:00.000Z"), - ImmutableMap.of("normal", 0) + ImmutableMap.of("normal", 0), + null ) ) ) @@ -768,7 +785,8 @@ public class RunRulesTest Collections.singletonList( new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"), - ImmutableMap.of("hot", 2) + ImmutableMap.of("hot", 2), + null ) ) ) @@ -847,7 +865,8 @@ public class RunRulesTest Collections.singletonList( new IntervalLoadRule( Intervals.of("2012-01-01/2013-01-01"), - ImmutableMap.of("hot", 1, DruidServer.DEFAULT_TIER, 1) + ImmutableMap.of("hot", 1, DruidServer.DEFAULT_TIER, 1), + null ) ) ) @@ -899,7 +918,8 @@ public class RunRulesTest Collections.singletonList( new IntervalLoadRule( Intervals.of("2012-01-01/2013-01-02"), - ImmutableMap.of("normal", 1) + ImmutableMap.of("normal", 1), + null ) ) ) @@ -982,7 +1002,7 @@ public class RunRulesTest mockEmptyPeon(); EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( - Collections.singletonList(new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1)))).atLeastOnce(); + Collections.singletonList(new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1), null))).atLeastOnce(); EasyMock.replay(databaseRuleManager); DruidCluster druidCluster = DruidCluster.builder().add( @@ -1019,7 +1039,7 @@ public class RunRulesTest EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( Collections.singletonList( - new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 3)) + new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 3), null) )).atLeastOnce(); EasyMock.replay(databaseRuleManager); @@ -1081,7 +1101,7 @@ public class RunRulesTest EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( Collections.singletonList( - new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1)) + new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1), null) )).atLeastOnce(); EasyMock.replay(databaseRuleManager); @@ -1133,7 +1153,8 @@ public class RunRulesTest EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( Collections.singletonList( new ForeverLoadRule( - ImmutableMap.of(DruidServer.DEFAULT_TIER, numReplicants) + ImmutableMap.of(DruidServer.DEFAULT_TIER, numReplicants), + null ) )).atLeastOnce(); EasyMock.replay(databaseRuleManager); @@ -1194,7 +1215,8 @@ public class RunRulesTest EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( Collections.singletonList( new ForeverLoadRule( - ImmutableMap.of(DruidServer.DEFAULT_TIER, numReplicants) + ImmutableMap.of(DruidServer.DEFAULT_TIER, numReplicants), + null ) )).atLeastOnce(); EasyMock.replay(databaseRuleManager); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java index 8faa4d361d0..83aa11fb303 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java @@ -316,7 +316,8 @@ public class UnloadUnusedSegmentsTest ImmutableMap.of( DruidServer.DEFAULT_TIER, 1, "tier2", 1 - ) + ), + null ) )).anyTimes(); @@ -326,7 +327,8 @@ public class UnloadUnusedSegmentsTest ImmutableMap.of( DruidServer.DEFAULT_TIER, 1, "tier2", 1 - ) + ), + null ) )).anyTimes(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/ForeverLoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/ForeverLoadRuleTest.java index c9be7f63ddb..c1497afe899 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/ForeverLoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/ForeverLoadRuleTest.java @@ -22,8 +22,10 @@ package org.apache.druid.server.coordinator.rules; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.client.DruidServer; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.IAE; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; @@ -32,12 +34,12 @@ import java.util.Map; public class ForeverLoadRuleTest { + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + @Test - public void testSerdeNullTieredReplicants() throws Exception + public void testSerde() throws Exception { - ForeverLoadRule rule = new ForeverLoadRule( - null - ); + ForeverLoadRule rule = new ForeverLoadRule(null, null); ObjectMapper jsonMapper = new DefaultObjectMapper(); Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); @@ -47,45 +49,79 @@ public class ForeverLoadRuleTest } @Test - public void testMappingNullTieredReplicants() throws Exception + public void testCreatingNegativeTieredReplicants() { - String inputJson = "{\n" - + " \"type\": \"loadForever\"\n" - + "}"; - String expectedJson = " {\n" - + " \"tieredReplicants\": {\n" - + " \"" + DruidServer.DEFAULT_TIER + "\": " + DruidServer.DEFAULT_NUM_REPLICANTS + "\n" - + " },\n" - + " \"type\": \"loadForever\"\n" - + " }"; - ObjectMapper jsonMapper = new DefaultObjectMapper(); - ForeverLoadRule inputForeverLoadRule = jsonMapper.readValue(inputJson, ForeverLoadRule.class); - ForeverLoadRule expectedForeverLoadRule = jsonMapper.readValue(expectedJson, ForeverLoadRule.class); - Assert.assertEquals(expectedForeverLoadRule.getTieredReplicants(), inputForeverLoadRule.getTieredReplicants()); + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> + new ForeverLoadRule( + ImmutableMap.of(DruidServer.DEFAULT_TIER, -1), + null + ) + ), + DruidExceptionMatcher.invalidInput().expectMessageContains( + "Invalid number of replicas for tier [_default_tier]. Value [-1] must be positive." + ) + ); } - @Test(expected = IAE.class) + @Test public void testEmptyTieredReplicants() throws Exception { - ForeverLoadRule rule = new ForeverLoadRule( - ImmutableMap.of() - ); + ForeverLoadRule rule = new ForeverLoadRule(ImmutableMap.of(), false); - ObjectMapper jsonMapper = new DefaultObjectMapper(); - Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); + LoadRule reread = (LoadRule) OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsString(rule), Rule.class); + Assert.assertEquals(ImmutableMap.of(), reread.getTieredReplicants()); } - @Test(expected = IAE.class) - public void testEmptyReplicantValue() throws Exception + @Test + public void testNullReplicantValue() { // Immutable map does not allow null values Map tieredReplicants = new HashMap<>(); tieredReplicants.put("tier", null); - ForeverLoadRule rule = new ForeverLoadRule( - tieredReplicants - ); - ObjectMapper jsonMapper = new DefaultObjectMapper(); - Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> + new ForeverLoadRule( + tieredReplicants, + true + ) + ), + DruidExceptionMatcher.invalidInput().expectMessageContains( + "Invalid number of replicas for tier [tier]. Value must not be null." + ) + ); + } + + @Test + public void testShouldCreateDefaultTier() throws Exception + { + String inputJson = " {\n" + + " \"type\": \"loadForever\"\n" + + " }"; + ForeverLoadRule inputForeverLoadRule = OBJECT_MAPPER.readValue(inputJson, ForeverLoadRule.class); + Assert.assertEquals(ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), inputForeverLoadRule.getTieredReplicants()); + } + + @Test + public void testUseDefaultTierAsTrueShouldCreateDefaultTier() throws Exception + { + String inputJson = " {\n" + + " \"type\": \"loadForever\"\n," + + " \"useDefaultTierForNull\": \"true\"\n" + + " }"; + ForeverLoadRule inputForeverLoadRule = OBJECT_MAPPER.readValue(inputJson, ForeverLoadRule.class); + Assert.assertEquals(ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), inputForeverLoadRule.getTieredReplicants()); + } + + @Test + public void testUseDefaultTierAsFalseShouldCreateEmptyMap() throws Exception + { + String inputJson = " {\n" + + " \"type\": \"loadForever\"\n," + + " \"useDefaultTierForNull\": \"false\"\n" + + " }"; + ForeverLoadRule inputForeverLoadRule = OBJECT_MAPPER.readValue(inputJson, ForeverLoadRule.class); + Assert.assertEquals(ImmutableMap.of(), inputForeverLoadRule.getTieredReplicants()); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/IntervalLoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/IntervalLoadRuleTest.java index 52e892dd8cf..5eae342d2ed 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/IntervalLoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/IntervalLoadRuleTest.java @@ -22,25 +22,34 @@ package org.apache.druid.server.coordinator.rules; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.client.DruidServer; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; +import java.util.HashMap; +import java.util.Map; + /** + * Unit tests for {@link IntervalLoadRule} */ public class IntervalLoadRuleTest { + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + @Test public void testSerde() throws Exception { IntervalLoadRule rule = new IntervalLoadRule( Intervals.of("0/3000"), - ImmutableMap.of(DruidServer.DEFAULT_TIER, 2) + ImmutableMap.of(DruidServer.DEFAULT_TIER, 2), + null ); - ObjectMapper jsonMapper = new DefaultObjectMapper(); - Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); + Rule reread = OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsString(rule), Rule.class); Assert.assertEquals(rule, reread); } @@ -49,36 +58,85 @@ public class IntervalLoadRuleTest public void testSerdeNullTieredReplicants() throws Exception { IntervalLoadRule rule = new IntervalLoadRule( - Intervals.of("0/3000"), null + Intervals.of("0/3000"), null, false ); - ObjectMapper jsonMapper = new DefaultObjectMapper(); - Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); + Rule reread = OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsString(rule), Rule.class); Assert.assertEquals(rule, reread); - Assert.assertEquals( - ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), - rule.getTieredReplicants() + Assert.assertEquals(ImmutableMap.of(), rule.getTieredReplicants()); + } + + @Test + public void testCreatingNegativeTieredReplicants() + { + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> + new IntervalLoadRule( + Intervals.of("0/3000"), + ImmutableMap.of(DruidServer.DEFAULT_TIER, -1), + null + ) + ), + DruidExceptionMatcher.invalidInput().expectMessageContains( + "Invalid number of replicas for tier [_default_tier]. Value [-1] must be positive." + ) ); } @Test - public void testMappingNullTieredReplicants() throws Exception + public void testNullReplicantValue() { - String inputJson = " {\n" + // Immutable map does not allow null values + Map tieredReplicants = new HashMap<>(); + tieredReplicants.put("tier", null); + + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> + new IntervalLoadRule( + Intervals.of("0/3000"), + tieredReplicants, + null + ) + ), + DruidExceptionMatcher.invalidInput().expectMessageContains( + "Invalid number of replicas for tier [tier]. Value must not be null." + ) + ); + } + + @Test + public void testShouldCreateDefaultTier() throws Exception + { + String inputJson = " {\n" + " \"interval\": \"0000-01-01T00:00:00.000-05:50:36/3000-01-01T00:00:00.000-06:00\",\n" + " \"type\": \"loadByInterval\"\n" - + " }"; - String expectedJson = "{\n" - + " \"interval\": \"0000-01-01T00:00:00.000-05:50:36/3000-01-01T00:00:00.000-06:00\",\n" - + " \"tieredReplicants\": {\n" - + " \"" + DruidServer.DEFAULT_TIER + "\": " + DruidServer.DEFAULT_NUM_REPLICANTS + "\n" - + " },\n" - + " \"type\": \"loadByInterval\"\n" - + " }"; - ObjectMapper jsonMapper = new DefaultObjectMapper(); - IntervalLoadRule inputIntervalLoadRule = jsonMapper.readValue(inputJson, IntervalLoadRule.class); - IntervalLoadRule expectedIntervalLoadRule = jsonMapper.readValue(expectedJson, IntervalLoadRule.class); - Assert.assertEquals(expectedIntervalLoadRule, inputIntervalLoadRule); + + " }"; + IntervalLoadRule inputIntervalLoadRule = OBJECT_MAPPER.readValue(inputJson, IntervalLoadRule.class); + Assert.assertEquals(ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), inputIntervalLoadRule.getTieredReplicants()); + } + + @Test + public void testUseDefaultTierAsTrueShouldCreateDefaultTier() throws Exception + { + String inputJson = " {\n" + + " \"interval\": \"0000-01-01T00:00:00.000-05:50:36/3000-01-01T00:00:00.000-06:00\",\n" + + " \"type\": \"loadByInterval\",\n" + + " \"useDefaultTierForNull\": \"true\"\n" + + " }"; + IntervalLoadRule inputIntervalLoadRule = OBJECT_MAPPER.readValue(inputJson, IntervalLoadRule.class); + Assert.assertEquals(ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), inputIntervalLoadRule.getTieredReplicants()); + } + + @Test + public void testUseDefaultTierAsFalseShouldCreateEmptyMap() throws Exception + { + String inputJson = " {\n" + + " \"interval\": \"0000-01-01T00:00:00.000-05:50:36/3000-01-01T00:00:00.000-06:00\",\n" + + " \"type\": \"loadByInterval\",\n" + + " \"useDefaultTierForNull\": \"false\"\n" + + " }"; + IntervalLoadRule inputIntervalLoadRule = OBJECT_MAPPER.readValue(inputJson, IntervalLoadRule.class); + Assert.assertEquals(ImmutableMap.of(), inputIntervalLoadRule.getTieredReplicants()); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java index 85aeeb77db1..668013f7ff3 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.DateTimes; @@ -705,7 +706,7 @@ public class LoadRuleTest private static LoadRule loadForever(final Map tieredReplicants) { - return new ForeverLoadRule(tieredReplicants); + return new ForeverLoadRule(tieredReplicants, null); } private static LoadQueuePeon createEmptyPeon() @@ -764,4 +765,13 @@ public class LoadRuleTest static final String T1 = "tier1"; static final String T2 = "tier2"; } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(LoadRule.class) + .withNonnullFields("tieredReplicants") + .usingGetClass() + .verify(); + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java index 84b72614000..7b1dd2085f0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java @@ -21,22 +21,31 @@ package org.apache.druid.server.coordinator.rules; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.client.DruidServer; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; +import org.hamcrest.MatcherAssert; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; +import java.util.HashMap; +import java.util.Map; + /** */ public class PeriodLoadRuleTest { + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + private static final DataSegment.Builder BUILDER = DataSegment .builder() .dataSource("test") @@ -51,7 +60,8 @@ public class PeriodLoadRuleTest PeriodLoadRule rule = new PeriodLoadRule( new Period("P5000Y"), false, - ImmutableMap.of("", 0) + ImmutableMap.of("", 0), + null ); Assert.assertTrue(rule.appliesTo(BUILDER.interval(Intervals.of("2012-01-01/2012-12-31")).build(), now)); @@ -66,7 +76,8 @@ public class PeriodLoadRuleTest PeriodLoadRule rule = new PeriodLoadRule( new Period("P1M"), false, - ImmutableMap.of("", 0) + ImmutableMap.of("", 0), + null ); Assert.assertTrue(rule.appliesTo(BUILDER.interval(new Interval(now.minusWeeks(1), now)).build(), now)); @@ -80,7 +91,7 @@ public class PeriodLoadRuleTest Assert.assertFalse( rule.appliesTo( BUILDER.interval(new Interval(now.plusDays(1), now.plusDays(2))) - .build(), + .build(), now ) ); @@ -91,24 +102,25 @@ public class PeriodLoadRuleTest { DateTime now = DateTimes.of("2012-12-31T01:00:00"); PeriodLoadRule rule = new PeriodLoadRule( - new Period("P1M"), - false, - ImmutableMap.of("", 0) + new Period("P1M"), + false, + ImmutableMap.of("", 0), + null ); Assert.assertTrue( - rule.appliesTo( - BUILDER.interval(new Interval(now.minusWeeks(1), now.plusWeeks(1))).build(), - now - ) + rule.appliesTo( + BUILDER.interval(new Interval(now.minusWeeks(1), now.plusWeeks(1))).build(), + now + ) ); Assert.assertTrue( - rule.appliesTo( - BUILDER.interval( - new Interval(now.minusMonths(1).minusWeeks(1), now.minusMonths(1).plusWeeks(1)) - ).build(), - now - ) + rule.appliesTo( + BUILDER.interval( + new Interval(now.minusMonths(1).minusWeeks(1), now.minusMonths(1).plusWeeks(1)) + ).build(), + now + ) ); } @@ -119,12 +131,14 @@ public class PeriodLoadRuleTest PeriodLoadRule includeFutureRule = new PeriodLoadRule( new Period("P2D"), true, - ImmutableMap.of("", 0) + ImmutableMap.of("", 0), + null ); PeriodLoadRule notIncludeFutureRule = new PeriodLoadRule( new Period("P2D"), false, - ImmutableMap.of("", 0) + ImmutableMap.of("", 0), + null ); Assert.assertTrue( @@ -148,11 +162,10 @@ public class PeriodLoadRuleTest public void testSerdeNull() throws Exception { PeriodLoadRule rule = new PeriodLoadRule( - new Period("P1D"), null, null + new Period("P1D"), null, null, null ); - ObjectMapper jsonMapper = new DefaultObjectMapper(); - Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); + Rule reread = OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsString(rule), Rule.class); Assert.assertEquals(rule.getPeriod(), ((PeriodLoadRule) reread).getPeriod()); Assert.assertEquals(rule.isIncludeFuture(), ((PeriodLoadRule) reread).isIncludeFuture()); @@ -174,16 +187,100 @@ public class PeriodLoadRuleTest String expectedJson = "{\n" + " \"period\": \"P1D\",\n" + " \"includeFuture\": " + PeriodLoadRule.DEFAULT_INCLUDE_FUTURE + ",\n" - + " \"tieredReplicants\": {\n" - + " \"" + DruidServer.DEFAULT_TIER + "\": " + DruidServer.DEFAULT_NUM_REPLICANTS + "\n" - + " },\n" + " \"type\": \"loadByPeriod\"\n" + " }"; - ObjectMapper jsonMapper = new DefaultObjectMapper(); - PeriodLoadRule inputPeriodLoadRule = jsonMapper.readValue(inputJson, PeriodLoadRule.class); - PeriodLoadRule expectedPeriodLoadRule = jsonMapper.readValue(expectedJson, PeriodLoadRule.class); + PeriodLoadRule inputPeriodLoadRule = OBJECT_MAPPER.readValue(inputJson, PeriodLoadRule.class); + PeriodLoadRule expectedPeriodLoadRule = OBJECT_MAPPER.readValue(expectedJson, PeriodLoadRule.class); Assert.assertEquals(expectedPeriodLoadRule.getTieredReplicants(), inputPeriodLoadRule.getTieredReplicants()); Assert.assertEquals(expectedPeriodLoadRule.getPeriod(), inputPeriodLoadRule.getPeriod()); Assert.assertEquals(expectedPeriodLoadRule.isIncludeFuture(), inputPeriodLoadRule.isIncludeFuture()); } + + @Test + public void testCreatingNegativeTieredReplicants() + { + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> + new PeriodLoadRule( + Period.days(1), + true, + ImmutableMap.of(DruidServer.DEFAULT_TIER, -1), + true + ) + ), + DruidExceptionMatcher.invalidInput().expectMessageContains( + "Invalid number of replicas for tier [_default_tier]. Value [-1] must be positive." + ) + ); + } + + @Test + public void testNullReplicantValue() + { + // Immutable map does not allow null values + Map tieredReplicants = new HashMap<>(); + tieredReplicants.put("tier", null); + + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> + new PeriodLoadRule( + Period.days(1), + true, + tieredReplicants, + true + ) + ), + DruidExceptionMatcher.invalidInput().expectMessageContains( + "Invalid number of replicas for tier [tier]. Value must not be null." + ) + ); + } + + + @Test + public void testShouldCreateDefaultTier() throws Exception + { + String inputJson = " {\n" + + " \"period\": \"P1D\",\n" + + " \"includeFuture\": " + PeriodLoadRule.DEFAULT_INCLUDE_FUTURE + ",\n" + + " \"type\": \"loadByPeriod\"\n" + + " }"; + PeriodLoadRule inputPeriodLoadRule = OBJECT_MAPPER.readValue(inputJson, PeriodLoadRule.class); + Assert.assertEquals(ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), inputPeriodLoadRule.getTieredReplicants()); + } + + @Test + public void testUseDefaultTierAsTrueShouldCreateDefaultTier() throws Exception + { + String inputJson = " {\n" + + " \"period\": \"P1D\",\n" + + " \"includeFuture\": " + PeriodLoadRule.DEFAULT_INCLUDE_FUTURE + ",\n" + + " \"useDefaultTierForNull\": \"true\",\n" + + " \"type\": \"loadByPeriod\"\n" + + " }"; + PeriodLoadRule inputPeriodLoadRule = OBJECT_MAPPER.readValue(inputJson, PeriodLoadRule.class); + Assert.assertEquals(ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), inputPeriodLoadRule.getTieredReplicants()); + } + + @Test + public void testUseDefaultTierAsFalseShouldCreateEmptyMap() throws Exception + { + String inputJson = " {\n" + + " \"period\": \"P1D\",\n" + + " \"includeFuture\": " + PeriodLoadRule.DEFAULT_INCLUDE_FUTURE + ",\n" + + " \"useDefaultTierForNull\": \"false\",\n" + + " \"type\": \"loadByPeriod\"\n" + + " }"; + PeriodLoadRule inputPeriodLoadRule = OBJECT_MAPPER.readValue(inputJson, PeriodLoadRule.class); + Assert.assertEquals(ImmutableMap.of(), inputPeriodLoadRule.getTieredReplicants()); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(PeriodLoadRule.class) + .withNonnullFields("tieredReplicants") + .usingGetClass() + .verify(); + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java index 99d4d9d0a83..e64ab0bcc46 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java @@ -276,7 +276,7 @@ public abstract class CoordinatorSimulationBaseTest implements Rule forever() { - return new ForeverLoadRule(tieredReplicants); + return new ForeverLoadRule(tieredReplicants, null); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestMetadataRuleManager.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestMetadataRuleManager.java index 9ca037b0cfb..9bb877112cf 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestMetadataRuleManager.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestMetadataRuleManager.java @@ -40,7 +40,7 @@ public class TestMetadataRuleManager implements MetadataRuleManager { rules.put( DEFAULT_DATASOURCE, - Collections.singletonList(new ForeverLoadRule(null)) + Collections.singletonList(new ForeverLoadRule(null, null)) ); } diff --git a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java index 69200382f35..4e59e913e23 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java @@ -631,7 +631,7 @@ public class DataSourcesResourceTest public void testIsHandOffComplete() { MetadataRuleManager databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); - Rule loadRule = new IntervalLoadRule(Intervals.of("2013-01-02T00:00:00Z/2013-01-03T00:00:00Z"), null); + Rule loadRule = new IntervalLoadRule(Intervals.of("2013-01-02T00:00:00Z/2013-01-03T00:00:00Z"), null, null); Rule dropRule = new IntervalDropRule(Intervals.of("2013-01-01T00:00:00Z/2013-01-02T00:00:00Z")); DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, databaseRuleManager, null, null, null); diff --git a/services/src/test/java/org/apache/druid/server/router/CoordinatorRuleManagerTest.java b/services/src/test/java/org/apache/druid/server/router/CoordinatorRuleManagerTest.java index 89741c0c706..553b6ef7b27 100644 --- a/services/src/test/java/org/apache/druid/server/router/CoordinatorRuleManagerTest.java +++ b/services/src/test/java/org/apache/druid/server/router/CoordinatorRuleManagerTest.java @@ -49,7 +49,7 @@ public class CoordinatorRuleManagerTest private static final String DATASOURCE1 = "datasource1"; private static final String DATASOURCE2 = "datasource2"; private static final List DEFAULT_RULES = ImmutableList.of( - new ForeverLoadRule(ImmutableMap.of("__default", 2)) + new ForeverLoadRule(ImmutableMap.of("__default", 2), null) ); @org.junit.Rule @@ -109,7 +109,7 @@ public class CoordinatorRuleManagerTest manager.poll(); final List rules = manager.getRulesWithDefault(DATASOURCE2); final List expectedRules = new ArrayList<>(); - expectedRules.add(new ForeverLoadRule(null)); + expectedRules.add(new ForeverLoadRule(null, null)); expectedRules.add(new IntervalDropRule(Intervals.of("2020-01-01/2020-01-02"))); expectedRules.addAll(DEFAULT_RULES); Assert.assertEquals(expectedRules, rules); @@ -119,16 +119,16 @@ public class CoordinatorRuleManagerTest { final Map> rules = ImmutableMap.of( DATASOURCE1, - ImmutableList.of(new ForeverLoadRule(null)), + ImmutableList.of(new ForeverLoadRule(null, null)), DATASOURCE2, - ImmutableList.of(new ForeverLoadRule(null), new IntervalDropRule(Intervals.of("2020-01-01/2020-01-02"))), + ImmutableList.of(new ForeverLoadRule(null, null), new IntervalDropRule(Intervals.of("2020-01-01/2020-01-02"))), "datasource3", ImmutableList.of( - new PeriodLoadRule(new Period("P1M"), true, null), + new PeriodLoadRule(new Period("P1M"), true, null, null), new ForeverDropRule() ), TieredBrokerConfig.DEFAULT_RULE_NAME, - ImmutableList.of(new ForeverLoadRule(ImmutableMap.of("__default", 2))) + ImmutableList.of(new ForeverLoadRule(ImmutableMap.of("__default", 2), null)) ); final StringFullResponseHolder holder = EasyMock.niceMock(StringFullResponseHolder.class); EasyMock.expect(holder.getStatus()) diff --git a/services/src/test/java/org/apache/druid/server/router/TieredBrokerHostSelectorTest.java b/services/src/test/java/org/apache/druid/server/router/TieredBrokerHostSelectorTest.java index 670b61702e2..8dbf0329236 100644 --- a/services/src/test/java/org/apache/druid/server/router/TieredBrokerHostSelectorTest.java +++ b/services/src/test/java/org/apache/druid/server/router/TieredBrokerHostSelectorTest.java @@ -418,11 +418,12 @@ public class TieredBrokerHostSelectorTest public List getRulesWithDefault(String dataSource) { return Arrays.asList( - new IntervalLoadRule(Intervals.of("2013/2014"), ImmutableMap.of("hot", 1)), - new IntervalLoadRule(Intervals.of("2012/2013"), ImmutableMap.of("medium", 1)), + new IntervalLoadRule(Intervals.of("2013/2014"), ImmutableMap.of("hot", 1), null), + new IntervalLoadRule(Intervals.of("2012/2013"), ImmutableMap.of("medium", 1), null), new IntervalLoadRule( Intervals.of("2011/2012"), - ImmutableMap.of(DruidServer.DEFAULT_TIER, 1) + ImmutableMap.of(DruidServer.DEFAULT_TIER, 1), + null ) ); } From 31b9d5695d8db387f6422a4e334b43dba4cee6e1 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 22 Jun 2023 02:31:50 -0700 Subject: [PATCH 15/74] Extend InitializedNullHandlingTest instead of NullHandlingTest (#14467) NullHandlingTest is an actual test, it shouldn't be used as a base class --- .../druid/indexing/input/DruidSegmentReaderTest.java | 4 ++-- .../druid/indexing/input/InputRowSchemasTest.java | 4 ++-- .../apache/druid/common/config/NullHandlingTest.java | 10 ++-------- .../druid/query/scan/MultiSegmentScanQueryTest.java | 4 ++-- .../druid/segment/join/JoinableFactoryWrapperTest.java | 4 ++-- .../druid/segment/join/lookup/LookupJoinableTest.java | 4 ++-- 6 files changed, 12 insertions(+), 18 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java index e2bae93e675..5580bfc5cc0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.commons.lang.mutable.MutableBoolean; -import org.apache.druid.common.config.NullHandlingTest; import org.apache.druid.data.input.BytesCountingInputEntity; import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputEntity; @@ -60,6 +59,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.joda.time.Interval; @@ -80,7 +80,7 @@ import java.util.Map; import static org.junit.Assert.assertThrows; -public class DruidSegmentReaderTest extends NullHandlingTest +public class DruidSegmentReaderTest extends InitializedNullHandlingTest { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java index 991a5950f9a..44850ad0558 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java @@ -21,7 +21,6 @@ package org.apache.druid.indexing.input; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import org.apache.druid.common.config.NullHandlingTest; import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -40,12 +39,13 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; import java.util.Arrays; -public class InputRowSchemasTest extends NullHandlingTest +public class InputRowSchemasTest extends InitializedNullHandlingTest { @Test public void test_createColumnsFilter_normal() diff --git a/processing/src/test/java/org/apache/druid/common/config/NullHandlingTest.java b/processing/src/test/java/org/apache/druid/common/config/NullHandlingTest.java index 3e1a1a87c3d..8bfb36d25d3 100644 --- a/processing/src/test/java/org/apache/druid/common/config/NullHandlingTest.java +++ b/processing/src/test/java/org/apache/druid/common/config/NullHandlingTest.java @@ -19,21 +19,15 @@ package org.apache.druid.common.config; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; -import org.junit.BeforeClass; import org.junit.Test; import static org.apache.druid.common.config.NullHandling.replaceWithDefault; import static org.junit.Assert.assertEquals; -public class NullHandlingTest +public final class NullHandlingTest extends InitializedNullHandlingTest { - @BeforeClass - public static void setUpClass() - { - NullHandling.initializeForTests(); - } - @Test public void test_defaultValueForClass_float() { diff --git a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java index fe8ff7a90d1..cc276c66328 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.Lists; import com.google.common.io.CharSource; import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; -import org.apache.druid.common.config.NullHandlingTest; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; @@ -47,6 +46,7 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.Interval; @@ -66,7 +66,7 @@ import java.util.List; * */ @RunWith(Parameterized.class) -public class MultiSegmentScanQueryTest extends NullHandlingTest +public class MultiSegmentScanQueryTest extends InitializedNullHandlingTest { private static final ScanQueryQueryToolChest TOOL_CHEST = new ScanQueryQueryToolChest( new ScanQueryConfig(), diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java index f4ee1676b59..0516287d33a 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.common.config.NullHandlingTest; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; @@ -41,6 +40,7 @@ import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.join.table.RowBasedIndexedTable; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -54,7 +54,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -public class JoinableFactoryWrapperTest extends NullHandlingTest +public class JoinableFactoryWrapperTest extends InitializedNullHandlingTest { public static final JoinableFactoryWrapper NOOP_JOINABLE_FACTORY_WRAPPER = new JoinableFactoryWrapper( NoopJoinableFactory.INSTANCE diff --git a/processing/src/test/java/org/apache/druid/segment/join/lookup/LookupJoinableTest.java b/processing/src/test/java/org/apache/druid/segment/join/lookup/LookupJoinableTest.java index 4261494ef55..ce1dc7fc8b4 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/lookup/LookupJoinableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/lookup/LookupJoinableTest.java @@ -22,11 +22,11 @@ package org.apache.druid.segment.join.lookup; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.common.config.NullHandlingTest; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.join.Joinable; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Before; import org.junit.Ignore; @@ -43,7 +43,7 @@ import java.util.Optional; import java.util.Set; @RunWith(MockitoJUnitRunner.class) -public class LookupJoinableTest extends NullHandlingTest +public class LookupJoinableTest extends InitializedNullHandlingTest { private static final String UNKNOWN_COLUMN = "UNKNOWN_COLUMN"; private static final String SEARCH_KEY_VALUE = "SEARCH_KEY_VALUE"; From 9b1779734b0bf59b790742b632a1761413482d03 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 22 Jun 2023 12:14:23 -0700 Subject: [PATCH 16/74] fix website mvn build (#14458) changes: * fix website mvn build * remove the i18n/en.json file add to gitignore * add spellcheck to mvn test phase --- .gitignore | 2 +- pom.xml | 4 - website/i18n/en.json | 790 ------------------------------------------- website/pom.xml | 15 +- 4 files changed, 14 insertions(+), 797 deletions(-) delete mode 100644 website/i18n/en.json diff --git a/.gitignore b/.gitignore index a60eb68173a..cd33e6271a1 100644 --- a/.gitignore +++ b/.gitignore @@ -35,7 +35,7 @@ integration-tests/gen-scripts/ **/.local/ **/druidapi.egg-info/ examples/quickstart/jupyter-notebooks/docker-jupyter/notebooks - +website/i18n/* # ignore NetBeans IDE specific files nbproject nbactions.xml diff --git a/pom.xml b/pom.xml index c495ea3255a..d8c08650db1 100644 --- a/pom.xml +++ b/pom.xml @@ -1726,10 +1726,6 @@ com.github.eirslett frontend-maven-plugin 1.11.3 - - ${node.version} - ${npm.version} - diff --git a/website/i18n/en.json b/website/i18n/en.json deleted file mode 100644 index 7e5249167e4..00000000000 --- a/website/i18n/en.json +++ /dev/null @@ -1,790 +0,0 @@ -{ - "_comment": "This file is auto-generated by write-translations.js", - "localized-strings": { - "next": "Next", - "previous": "Previous", - "tagline": "A fast analytical database", - "docs": { - "comparisons/druid-vs-elasticsearch": { - "title": "Apache Druid vs Elasticsearch" - }, - "comparisons/druid-vs-key-value": { - "title": "Apache Druid vs. Key/Value Stores (HBase/Cassandra/OpenTSDB)" - }, - "comparisons/druid-vs-kudu": { - "title": "Apache Druid vs Kudu" - }, - "comparisons/druid-vs-redshift": { - "title": "Apache Druid vs Redshift" - }, - "comparisons/druid-vs-spark": { - "title": "Apache Druid vs Spark" - }, - "comparisons/druid-vs-sql-on-hadoop": { - "title": "Apache Druid vs SQL-on-Hadoop" - }, - "configuration/human-readable-byte": { - "title": "Human-readable Byte Configuration Reference" - }, - "configuration/index": { - "title": "Configuration reference" - }, - "configuration/logging": { - "title": "Logging" - }, - "data-management/automatic-compaction": { - "title": "Automatic compaction" - }, - "data-management/compaction": { - "title": "Compaction" - }, - "data-management/delete": { - "title": "Data deletion" - }, - "data-management/index": { - "title": "Data management", - "sidebar_label": "Overview" - }, - "data-management/schema-changes": { - "title": "Schema changes" - }, - "data-management/update": { - "title": "Data updates" - }, - "dependencies/deep-storage": { - "title": "Deep storage" - }, - "dependencies/metadata-storage": { - "title": "Metadata storage" - }, - "dependencies/zookeeper": { - "title": "ZooKeeper" - }, - "design/architecture": { - "title": "Design" - }, - "design/auth": { - "title": "Authentication and Authorization" - }, - "design/broker": { - "title": "Broker" - }, - "design/coordinator": { - "title": "Coordinator Process" - }, - "design/extensions-contrib/dropwizard": { - "title": "Dropwizard metrics emitter" - }, - "design/historical": { - "title": "Historical Process" - }, - "design/index": { - "title": "Introduction to Apache Druid" - }, - "design/indexer": { - "title": "Indexer Process" - }, - "design/indexing-service": { - "title": "Indexing Service" - }, - "design/middlemanager": { - "title": "MiddleManager Process" - }, - "design/overlord": { - "title": "Overlord Process" - }, - "design/peons": { - "title": "Peons" - }, - "design/processes": { - "title": "Processes and servers" - }, - "design/router": { - "title": "Router Process" - }, - "design/segments": { - "title": "Segments" - }, - "development/build": { - "title": "Build from source" - }, - "development/experimental-features": { - "title": "Experimental features" - }, - "development/experimental": { - "title": "Experimental features" - }, - "development/extensions-contrib/aliyun-oss": { - "title": "Aliyun OSS" - }, - "development/extensions-contrib/ambari-metrics-emitter": { - "title": "Ambari Metrics Emitter" - }, - "development/extensions-contrib/cassandra": { - "title": "Apache Cassandra" - }, - "development/extensions-contrib/cloudfiles": { - "title": "Rackspace Cloud Files" - }, - "development/extensions-contrib/compressed-big-decimal": { - "title": "Compressed Big Decimal" - }, - "development/extensions-contrib/distinctcount": { - "title": "DistinctCount Aggregator" - }, - "development/extensions-contrib/gce-extensions": { - "title": "GCE Extensions" - }, - "development/extensions-contrib/graphite": { - "title": "Graphite Emitter" - }, - "development/extensions-contrib/influx": { - "title": "InfluxDB Line Protocol Parser" - }, - "development/extensions-contrib/influxdb-emitter": { - "title": "InfluxDB Emitter" - }, - "development/extensions-contrib/k8s-jobs": { - "title": "MM-less Druid in K8s" - }, - "development/extensions-contrib/kafka-emitter": { - "title": "Kafka Emitter" - }, - "development/extensions-contrib/materialized-view": { - "title": "Materialized View" - }, - "development/extensions-contrib/momentsketch-quantiles": { - "title": "Moment Sketches for Approximate Quantiles module" - }, - "development/extensions-contrib/moving-average-query": { - "title": "Moving Average Query" - }, - "development/extensions-contrib/opentsdb-emitter": { - "title": "OpenTSDB Emitter" - }, - "development/extensions-contrib/prometheus": { - "title": "Prometheus Emitter" - }, - "development/extensions-contrib/redis-cache": { - "title": "Druid Redis Cache" - }, - "development/extensions-contrib/sqlserver": { - "title": "Microsoft SQLServer" - }, - "development/extensions-contrib/statsd": { - "title": "StatsD Emitter" - }, - "development/extensions-contrib/tdigestsketch-quantiles": { - "title": "T-Digest Quantiles Sketch module" - }, - "development/extensions-contrib/thrift": { - "title": "Thrift" - }, - "development/extensions-contrib/time-min-max": { - "title": "Timestamp Min/Max aggregators" - }, - "development/extensions-core/approximate-histograms": { - "title": "Approximate Histogram aggregators" - }, - "development/extensions-core/avro": { - "title": "Apache Avro" - }, - "development/extensions-core/azure": { - "title": "Microsoft Azure" - }, - "development/extensions-core/bloom-filter": { - "title": "Bloom Filter" - }, - "development/extensions-core/datasketches-extension": { - "title": "DataSketches extension" - }, - "development/extensions-core/datasketches-hll": { - "title": "DataSketches HLL Sketch module" - }, - "development/extensions-core/datasketches-kll": { - "title": "DataSketches KLL Sketch module" - }, - "development/extensions-core/datasketches-quantiles": { - "title": "DataSketches Quantiles Sketch module" - }, - "development/extensions-core/datasketches-theta": { - "title": "DataSketches Theta Sketch module" - }, - "development/extensions-core/datasketches-tuple": { - "title": "DataSketches Tuple Sketch module" - }, - "development/extensions-core/druid-aws-rds": { - "title": "Druid AWS RDS Module" - }, - "development/extensions-core/druid-basic-security": { - "title": "Basic Security" - }, - "development/extensions-core/druid-kerberos": { - "title": "Kerberos" - }, - "development/extensions-core/druid-lookups": { - "title": "Cached Lookup Module" - }, - "development/extensions-core/druid-pac4j": { - "title": "Druid pac4j based Security extension" - }, - "development/extensions-core/druid-ranger-security": { - "title": "Apache Ranger Security" - }, - "development/extensions-core/examples": { - "title": "Extension Examples" - }, - "development/extensions-core/google": { - "title": "Google Cloud Storage" - }, - "development/extensions-core/hdfs": { - "title": "HDFS" - }, - "development/extensions-core/kafka-extraction-namespace": { - "title": "Apache Kafka Lookups" - }, - "development/extensions-core/kafka-ingestion": { - "title": "Apache Kafka ingestion", - "sidebar_label": "Apache Kafka ingestion" - }, - "development/extensions-core/kafka-supervisor-operations": { - "title": "Apache Kafka supervisor operations reference", - "sidebar_label": "Apache Kafka operations" - }, - "development/extensions-core/kafka-supervisor-reference": { - "title": "Apache Kafka supervisor reference", - "sidebar_label": "Apache Kafka supervisor" - }, - "development/extensions-core/kinesis-ingestion": { - "title": "Amazon Kinesis ingestion", - "sidebar_label": "Amazon Kinesis" - }, - "development/extensions-core/kubernetes": { - "title": "Kubernetes" - }, - "development/extensions-core/lookups-cached-global": { - "title": "Globally Cached Lookups" - }, - "development/extensions-core/mysql": { - "title": "MySQL Metadata Store" - }, - "development/extensions-core/orc": { - "title": "ORC Extension" - }, - "development/extensions-core/parquet": { - "title": "Apache Parquet Extension" - }, - "development/extensions-core/postgresql": { - "title": "PostgreSQL Metadata Store" - }, - "development/extensions-core/protobuf": { - "title": "Protobuf" - }, - "development/extensions-core/s3": { - "title": "S3-compatible" - }, - "development/extensions-core/simple-client-sslcontext": { - "title": "Simple SSLContext Provider Module" - }, - "development/extensions-core/stats": { - "title": "Stats aggregator" - }, - "development/extensions-core/test-stats": { - "title": "Test Stats Aggregators" - }, - "development/extensions": { - "title": "Extensions" - }, - "development/geo": { - "title": "Spatial filters" - }, - "development/javascript": { - "title": "JavaScript programming guide", - "sidebar_label": "JavaScript functionality" - }, - "development/modules": { - "title": "Creating extensions" - }, - "development/overview": { - "title": "Developing on Apache Druid", - "sidebar_label": "Developing on Druid" - }, - "development/versioning": { - "title": "Versioning" - }, - "ingestion/data-formats": { - "title": "Data formats" - }, - "ingestion/data-model": { - "title": "Druid data model", - "sidebar_label": "Data model" - }, - "ingestion/faq": { - "title": "Ingestion troubleshooting FAQ", - "sidebar_label": "Troubleshooting FAQ" - }, - "ingestion/hadoop": { - "title": "Hadoop-based ingestion", - "sidebar_label": "Hadoop-based" - }, - "ingestion/index": { - "title": "Ingestion" - }, - "ingestion/ingestion-spec": { - "title": "Ingestion spec reference", - "sidebar_label": "Ingestion spec" - }, - "ingestion/migrate-from-firehose": { - "title": "Migrate from firehose to input source ingestion", - "sidebar_label": "Migrate from firehose" - }, - "ingestion/native-batch-firehose": { - "title": "Native batch ingestion with firehose (Deprecated)", - "sidebar_label": "Firehose (deprecated)" - }, - "ingestion/native-batch-input-sources": { - "title": "Native batch input sources", - "sidebar_label": "Native batch: input sources" - }, - "ingestion/native-batch-simple-task": { - "title": "Native batch simple task indexing", - "sidebar_label": "Native batch (simple)" - }, - "ingestion/native-batch": { - "title": "Native batch ingestion", - "sidebar_label": "Native batch" - }, - "ingestion/partitioning": { - "title": "Partitioning", - "sidebar_label": "Partitioning" - }, - "ingestion/rollup": { - "title": "Data rollup", - "sidebar_label": "Data rollup" - }, - "ingestion/schema-design": { - "title": "Schema design tips" - }, - "ingestion/standalone-realtime": { - "title": "Realtime Process" - }, - "ingestion/tasks": { - "title": "Task reference" - }, - "ingestion/tranquility": { - "title": "Tranquility" - }, - "misc/math-expr": { - "title": "Expressions" - }, - "misc/papers-and-talks": { - "title": "Papers" - }, - "multi-stage-query/api": { - "title": "SQL-based ingestion and multi-stage query task API", - "sidebar_label": "API" - }, - "multi-stage-query/concepts": { - "title": "SQL-based ingestion concepts", - "sidebar_label": "Key concepts" - }, - "multi-stage-query/examples": { - "title": "SQL-based ingestion query examples", - "sidebar_label": "Examples" - }, - "multi-stage-query/index": { - "title": "SQL-based ingestion", - "sidebar_label": "Overview" - }, - "multi-stage-query/known-issues": { - "title": "SQL-based ingestion known issues", - "sidebar_label": "Known issues" - }, - "multi-stage-query/reference": { - "title": "SQL-based ingestion reference", - "sidebar_label": "Reference" - }, - "multi-stage-query/security": { - "title": "SQL-based ingestion security", - "sidebar_label": "Security" - }, - "operations/alerts": { - "title": "Alerts" - }, - "operations/api-reference": { - "title": "API reference" - }, - "operations/auth-ldap": { - "title": "Configure LDAP authentication", - "sidebar_label": "LDAP auth" - }, - "operations/basic-cluster-tuning": { - "title": "Basic cluster tuning" - }, - "operations/clean-metadata-store": { - "title": "Automated cleanup for metadata records", - "sidebar_label": "Automated metadata cleanup" - }, - "operations/deep-storage-migration": { - "title": "Deep storage migration" - }, - "operations/dump-segment": { - "title": "dump-segment tool" - }, - "operations/dynamic-config-provider": { - "title": "Dynamic Config Providers" - }, - "operations/export-metadata": { - "title": "Export Metadata Tool" - }, - "operations/getting-started": { - "title": "Getting started with Apache Druid" - }, - "operations/high-availability": { - "title": "High availability" - }, - "operations/http-compression": { - "title": "HTTP compression" - }, - "operations/insert-segment-to-db": { - "title": "insert-segment-to-db tool" - }, - "operations/java": { - "title": "Java runtime" - }, - "operations/kubernetes": { - "title": "kubernetes" - }, - "operations/metadata-migration": { - "title": "Metadata Migration" - }, - "operations/metrics": { - "title": "Metrics" - }, - "operations/mixed-workloads": { - "title": "Configure Druid for mixed workloads", - "sidebar_label": "Mixed workloads" - }, - "operations/other-hadoop": { - "title": "Working with different versions of Apache Hadoop" - }, - "operations/password-provider": { - "title": "Password providers" - }, - "operations/pull-deps": { - "title": "pull-deps tool" - }, - "operations/python": { - "title": "Python Installation" - }, - "operations/request-logging": { - "title": "Request logging", - "sidebar_label": "Request logging" - }, - "operations/reset-cluster": { - "title": "reset-cluster tool" - }, - "operations/rolling-updates": { - "title": "Rolling updates" - }, - "operations/rule-configuration": { - "title": "Using rules to drop and retain data" - }, - "operations/security-overview": { - "title": "Security overview" - }, - "operations/security-user-auth": { - "title": "User authentication and authorization" - }, - "operations/segment-optimization": { - "title": "Segment size optimization" - }, - "operations/single-server": { - "title": "Single server deployment" - }, - "operations/tls-support": { - "title": "TLS support" - }, - "operations/use_sbt_to_build_fat_jar": { - "title": "Content for build.sbt" - }, - "operations/web-console": { - "title": "Web console" - }, - "querying/aggregations": { - "title": "Aggregations" - }, - "querying/caching": { - "title": "Query caching" - }, - "querying/datasource": { - "title": "Datasources" - }, - "querying/datasourcemetadataquery": { - "title": "DatasourceMetadata queries", - "sidebar_label": "DatasourceMetadata" - }, - "querying/dimensionspecs": { - "title": "Query dimensions", - "sidebar_label": "Dimensions" - }, - "querying/filters": { - "title": "Query filters", - "sidebar_label": "Filters" - }, - "querying/granularities": { - "title": "Query granularities", - "sidebar_label": "Granularities" - }, - "querying/groupbyquery": { - "title": "GroupBy queries", - "sidebar_label": "GroupBy" - }, - "querying/having": { - "title": "Having filters (groupBy)" - }, - "querying/hll-old": { - "title": "Cardinality/HyperUnique aggregators" - }, - "querying/joins": { - "title": "Joins" - }, - "querying/limitspec": { - "title": "Sorting and limiting (groupBy)" - }, - "querying/lookups": { - "title": "Lookups" - }, - "querying/multi-value-dimensions": { - "title": "Multi-value dimensions" - }, - "querying/multitenancy": { - "title": "Multitenancy considerations", - "sidebar_label": "Multitenancy" - }, - "querying/nested-columns": { - "title": "Nested columns", - "sidebar_label": "Nested columns" - }, - "querying/post-aggregations": { - "title": "Post-aggregations" - }, - "querying/query-context": { - "title": "Query context", - "sidebar_label": "Query context" - }, - "querying/query-execution": { - "title": "Query execution" - }, - "querying/querying": { - "title": "Native queries" - }, - "querying/scan-query": { - "title": "Scan queries", - "sidebar_label": "Scan" - }, - "querying/searchquery": { - "title": "Search queries", - "sidebar_label": "Search" - }, - "querying/segmentmetadataquery": { - "title": "SegmentMetadata queries", - "sidebar_label": "SegmentMetadata" - }, - "querying/select-query": { - "title": "Select queries", - "sidebar_label": "Select" - }, - "querying/sorting-orders": { - "title": "String comparators" - }, - "querying/sql-aggregations": { - "title": "SQL aggregation functions", - "sidebar_label": "Aggregation functions" - }, - "querying/sql-api": { - "title": "Druid SQL API", - "sidebar_label": "Druid SQL API" - }, - "querying/sql-data-types": { - "title": "SQL data types", - "sidebar_label": "SQL data types" - }, - "querying/sql-functions": { - "title": "All Druid SQL functions", - "sidebar_label": "All functions" - }, - "querying/sql-jdbc": { - "title": "SQL JDBC driver API", - "sidebar_label": "JDBC driver API" - }, - "querying/sql-json-functions": { - "title": "SQL JSON functions", - "sidebar_label": "JSON functions" - }, - "querying/sql-metadata-tables": { - "title": "SQL metadata tables", - "sidebar_label": "SQL metadata tables" - }, - "querying/sql-multivalue-string-functions": { - "title": "SQL multi-value string functions", - "sidebar_label": "Multi-value string functions" - }, - "querying/sql-operators": { - "title": "Druid SQL Operators", - "sidebar_label": "Operators" - }, - "querying/sql-query-context": { - "title": "SQL query context", - "sidebar_label": "SQL query context" - }, - "querying/sql-scalar": { - "title": "SQL scalar functions", - "sidebar_label": "Scalar functions" - }, - "querying/sql-translation": { - "title": "SQL query translation", - "sidebar_label": "SQL query translation" - }, - "querying/sql": { - "title": "Druid SQL overview", - "sidebar_label": "Overview and syntax" - }, - "querying/timeboundaryquery": { - "title": "TimeBoundary queries", - "sidebar_label": "TimeBoundary" - }, - "querying/timeseriesquery": { - "title": "Timeseries queries", - "sidebar_label": "Timeseries" - }, - "querying/topnmetricspec": { - "title": "Sorting (topN)" - }, - "querying/topnquery": { - "title": "TopN queries", - "sidebar_label": "TopN" - }, - "querying/troubleshooting": { - "title": "Troubleshooting query execution in Druid", - "sidebar_label": "Troubleshooting" - }, - "querying/using-caching": { - "title": "Using query caching" - }, - "querying/virtual-columns": { - "title": "Virtual columns" - }, - "tutorials/cluster": { - "title": "Clustered deployment" - }, - "tutorials/docker": { - "title": "Tutorial: Run with Docker" - }, - "tutorials/index": { - "title": "Quickstart (local)" - }, - "tutorials/tutorial-batch-hadoop": { - "title": "Tutorial: Load batch data using Apache Hadoop", - "sidebar_label": "Load from Apache Hadoop" - }, - "tutorials/tutorial-batch-native": { - "title": "Load data with native batch ingestion" - }, - "tutorials/tutorial-batch": { - "title": "Tutorial: Loading a file", - "sidebar_label": "Load files natively" - }, - "tutorials/tutorial-compaction": { - "title": "Tutorial: Compacting segments", - "sidebar_label": "Compacting segments" - }, - "tutorials/tutorial-delete-data": { - "title": "Tutorial: Deleting data", - "sidebar_label": "Deleting data" - }, - "tutorials/tutorial-ingestion-spec": { - "title": "Tutorial: Writing an ingestion spec", - "sidebar_label": "Writing an ingestion spec" - }, - "tutorials/tutorial-jdbc": { - "title": "Tutorial: Using the JDBC driver to query Druid", - "sidebar_label": "JDBC connector" - }, - "tutorials/tutorial-jupyter-index": { - "title": "Jupyter Notebook tutorials" - }, - "tutorials/tutorial-kafka": { - "title": "Tutorial: Load streaming data from Apache Kafka", - "sidebar_label": "Load from Apache Kafka" - }, - "tutorials/tutorial-kerberos-hadoop": { - "title": "Configuring Apache Druid to use Kerberized Apache Hadoop as deep storage", - "sidebar_label": "Kerberized HDFS deep storage" - }, - "tutorials/tutorial-msq-convert-spec": { - "title": "Tutorial: Convert an ingestion spec for SQL-based ingestion", - "sidebar_label": "Convert ingestion spec to SQL" - }, - "tutorials/tutorial-msq-extern": { - "title": "Tutorial: Load files with SQL-based ingestion", - "sidebar_label": "Load files using SQL 🆕" - }, - "tutorials/tutorial-query": { - "title": "Tutorial: Querying data", - "sidebar_label": "Querying data" - }, - "tutorials/tutorial-retention": { - "title": "Tutorial: Configuring data retention", - "sidebar_label": "Configuring data retention" - }, - "tutorials/tutorial-rollup": { - "title": "Tutorial: Roll-up", - "sidebar_label": "Roll-up" - }, - "tutorials/tutorial-sketches-theta": { - "title": "Approximations with Theta sketches", - "sidebar_label": "Theta sketches" - }, - "tutorials/tutorial-transform-spec": { - "title": "Tutorial: Transforming input data", - "sidebar_label": "Transforming input data" - }, - "tutorials/tutorial-unnest-datasource": { - "title": "Tutorial: Unnest data in a column", - "sidebar_label": "Using the unnest datasource" - }, - "tutorials/tutorial-update-data": { - "title": "Tutorial: Updating existing data", - "sidebar_label": "Updating existing data" - } - }, - "links": { - "Technology": "Technology", - "Use Cases": "Use Cases", - "Powered By": "Powered By", - "Docs": "Docs", - "Community": "Community", - "Apache": "Apache", - "Download": "Download" - }, - "categories": { - "Getting started": "Getting started", - "Tutorials": "Tutorials", - "Design": "Design", - "Ingestion": "Ingestion", - "Data management": "Data management", - "Querying": "Querying", - "Configuration": "Configuration", - "Operations": "Operations", - "Development": "Development", - "Misc": "Misc", - "Hidden": "Hidden" - } - }, - "pages-strings": { - "Help Translate|recruit community translators for your project": "Help Translate", - "Edit this Doc|recruitment message asking to edit the doc source": "Edit", - "Translate this Doc|recruitment message asking to translate the docs": "Translate" - } -} diff --git a/website/pom.xml b/website/pom.xml index 10fcb92fed1..40af802eb6b 100644 --- a/website/pom.xml +++ b/website/pom.xml @@ -39,8 +39,8 @@ to stage documentation for an unreleased version --> ${project.parent.version} - v10.24.1 - 6.14.12 + v16.17.0 + 8.15.0 @@ -71,6 +71,17 @@ ${project.build.directory} + + spellcheck + + npm + + test + + run spellcheck + ${project.build.directory} + + From 7e2cf35d7b0485b5fe5abeb021c5811ee812b0c7 Mon Sep 17 00:00:00 2001 From: imply-cheddar <86940447+imply-cheddar@users.noreply.github.com> Date: Fri, 23 Jun 2023 17:15:32 +0900 Subject: [PATCH 17/74] Fix compatibility issue with SqlTaskResource (#14466) * Fix compatibility issue with SqlTaskResource The DruidException changes broke the response format for errors coming back from the SqlTaskResource, so fix those --- .../apache/druid/msq/sql/SqlTaskResource.java | 91 +++++++------------ .../apache/druid/msq/sql/SqlTaskStatus.java | 10 +- .../druid/msq/sql/SqlTaskStatusTest.java | 18 ++-- .../apache/druid/error/DruidException.java | 4 + .../apache/druid/query/BadQueryException.java | 2 - 5 files changed, 52 insertions(+), 73 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java index d270963db46..dd4ec4d2b3c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java @@ -23,22 +23,16 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.io.CountingOutputStream; import com.google.inject.Inject; -import org.apache.druid.common.exception.SanitizableException; import org.apache.druid.error.DruidException; import org.apache.druid.error.ErrorResponse; +import org.apache.druid.error.QueryExceptionCompat; import org.apache.druid.guice.annotations.MSQ; import org.apache.druid.indexer.TaskState; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.query.BadQueryException; -import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryException; -import org.apache.druid.query.QueryInterruptedException; -import org.apache.druid.query.QueryTimeoutException; -import org.apache.druid.query.QueryUnsupportedException; -import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.server.QueryResponse; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.security.Access; @@ -47,7 +41,6 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.DirectStatement; import org.apache.druid.sql.HttpStatement; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.SqlRowTransformer; import org.apache.druid.sql.SqlStatementFactory; import org.apache.druid.sql.http.ResultFormat; @@ -63,7 +56,6 @@ import javax.ws.rs.Produces; import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; import javax.ws.rs.core.StreamingOutput; import java.io.IOException; import java.util.Collections; @@ -162,43 +154,32 @@ public class SqlTaskResource } catch (DruidException e) { stmt.reporter().failed(e); - return Response.status(e.getStatusCode()) - .type(MediaType.APPLICATION_JSON_TYPE) - .entity(new ErrorResponse(e)) - .build(); + return buildNonOkResponse(sqlQueryId, e); } - // Kitchen-sinking the errors since they are all unchecked. - // Just copied from SqlResource. - catch (QueryCapacityExceededException cap) { - stmt.reporter().failed(cap); - return buildNonOkResponse(QueryCapacityExceededException.STATUS_CODE, cap, sqlQueryId); - } - catch (QueryUnsupportedException unsupported) { - stmt.reporter().failed(unsupported); - return buildNonOkResponse(QueryUnsupportedException.STATUS_CODE, unsupported, sqlQueryId); - } - catch (QueryTimeoutException timeout) { - stmt.reporter().failed(timeout); - return buildNonOkResponse(QueryTimeoutException.STATUS_CODE, timeout, sqlQueryId); - } - catch (SqlPlanningException | ResourceLimitExceededException e) { - stmt.reporter().failed(e); - return buildNonOkResponse(BadQueryException.STATUS_CODE, e, sqlQueryId); + catch (QueryException queryException) { + stmt.reporter().failed(queryException); + final DruidException underlyingException = DruidException.fromFailure(new QueryExceptionCompat(queryException)); + return buildNonOkResponse(sqlQueryId, underlyingException); } catch (ForbiddenException e) { - // No request logs for forbidden queries; same as SqlResource - throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() - .transformIfNeeded(e); // let ForbiddenExceptionMapper handle this + log.debug("Got forbidden request for reason [%s]", e.getErrorMessage()); + return buildNonOkResponse( + "forbidden", + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE) + ); } - // Calcite throws a java.lang.AssertionError which is type Error not Exception. Using Throwable catches both. - catch (Throwable e) { + // Calcite throws java.lang.AssertionError at various points in planning/validation. + catch (AssertionError | Exception e) { stmt.reporter().failed(e); log.noStackTrace().warn(e, "Failed to handle query: %s", sqlQueryId); return buildNonOkResponse( - Status.INTERNAL_SERVER_ERROR.getStatusCode(), - QueryInterruptedException.wrapIfNeeded(e), - sqlQueryId + sqlQueryId, + DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e.getMessage()) ); } finally { @@ -237,12 +218,16 @@ public class SqlTaskResource yielder.close(); if (taskId == null) { - // Note: no ID to include in error: that is the problem we're reporting. - return genericError( - Response.Status.INTERNAL_SERVER_ERROR, - "Internal error", - "Failed to issue query task", - null + // Note: no ID to include in error: that is the problem we're reporting. It would be really nice to know + // why we don't have an ID or more information about why things failed. Hopefully that gets returned to the + // user through a DruidExcpetion that makes it out of this code and this code never actually gets executed. + // Using a defensive exception just to report something with the opes that any time this actually happens, the + // fix is to make error reporting somewhere that actually understands more about why it failed. + return buildNonOkResponse( + null, + DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.DEFENSIVE) + .build("Failed to issue query task") ); } @@ -252,6 +237,7 @@ public class SqlTaskResource .build(); } + @SuppressWarnings("UnstableApiUsage") private Response buildStandardResponse( Sequence sequence, SqlQuery sqlQuery, @@ -316,22 +302,11 @@ public class SqlTaskResource } } - private Response buildNonOkResponse(int status, SanitizableException e, String sqlQueryId) - { - QueryException cleaned = (QueryException) serverConfig - .getErrorResponseTransformStrategy() - .transformIfNeeded(e); - return Response - .status(status) - .entity(new SqlTaskStatus(sqlQueryId, TaskState.FAILED, cleaned)) - .build(); - } - - private Response genericError(Response.Status status, String code, String msg, String id) + private Response buildNonOkResponse(String sqlQueryId, DruidException exception) { return Response - .status(status) - .entity(new SqlTaskStatus(id, TaskState.FAILED, new QueryException("FAILED", msg, null, null))) + .status(exception.getStatusCode()) + .entity(new SqlTaskStatus(sqlQueryId, TaskState.FAILED, new ErrorResponse(exception))) .build(); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java index d47533315e2..9fa14ee9944 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java @@ -23,8 +23,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.TaskState; -import org.apache.druid.query.QueryException; import javax.annotation.Nullable; import java.util.Objects; @@ -37,13 +37,13 @@ public class SqlTaskStatus private final String taskId; private final TaskState state; @Nullable - private final QueryException error; + private final ErrorResponse error; @JsonCreator public SqlTaskStatus( @JsonProperty("taskId") final String taskId, @JsonProperty("state") final TaskState state, - @JsonProperty("error") @Nullable final QueryException error + @JsonProperty("error") @Nullable final ErrorResponse error ) { this.taskId = Preconditions.checkNotNull(taskId, "taskId"); @@ -66,7 +66,7 @@ public class SqlTaskStatus @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public QueryException getError() + public ErrorResponse getError() { return error; } @@ -98,7 +98,7 @@ public class SqlTaskStatus return "SqlTaskStatus{" + "taskId='" + taskId + '\'' + ", state=" + state + - ", error=" + error + + ", error=" + (error == null ? "null" : error.getAsMap()) + '}'; } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java index 03efc96e6de..301f91ce8d3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java @@ -21,8 +21,9 @@ package org.apache.druid.msq.sql; import com.fasterxml.jackson.databind.ObjectMapper; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.TaskState; -import org.apache.druid.query.QueryException; import org.apache.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Test; @@ -34,22 +35,23 @@ public class SqlTaskStatusTest { final ObjectMapper mapper = TestHelper.makeJsonMapper(); + DruidException underlyingException = DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("error message"); + final SqlTaskStatus status = new SqlTaskStatus( "taskid", TaskState.FAILED, - new QueryException( - "error code", - "error message", - "error class", - "host" - ) + new ErrorResponse(underlyingException) ); final SqlTaskStatus status2 = mapper.readValue(mapper.writeValueAsString(status), SqlTaskStatus.class); Assert.assertEquals(status.getTaskId(), status2.getTaskId()); Assert.assertEquals(status.getState(), status2.getState()); - Assert.assertEquals(status.getError().getErrorCode(), status2.getError().getErrorCode()); + Assert.assertNotNull(status.getError()); + Assert.assertNotNull(status2.getError()); + Assert.assertEquals(status.getError().getAsMap(), status2.getError().getAsMap()); } @Test diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index 6acedf55fdb..81a7939ec4a 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -315,6 +315,10 @@ public class DruidException extends RuntimeException * Means that the error is a problem with authorization. */ UNAUTHORIZED(401), + /** + * Means that an action that was attempted is forbidden + */ + FORBIDDEN(403), /** * Means that some capacity limit was exceeded, this could be due to throttling or due to some system limit */ diff --git a/processing/src/main/java/org/apache/druid/query/BadQueryException.java b/processing/src/main/java/org/apache/druid/query/BadQueryException.java index e627c966ade..dc498e85e91 100644 --- a/processing/src/main/java/org/apache/druid/query/BadQueryException.java +++ b/processing/src/main/java/org/apache/druid/query/BadQueryException.java @@ -26,8 +26,6 @@ package org.apache.druid.query; */ public abstract class BadQueryException extends QueryException { - public static final int STATUS_CODE = 400; - protected BadQueryException(String errorCode, String errorMessage, String errorClass) { this(errorCode, errorMessage, errorClass, null); From b6d6e3b827878c61516c302e36f9c0cf189b6248 Mon Sep 17 00:00:00 2001 From: Peter Marshall Date: Fri, 23 Jun 2023 09:37:24 +0100 Subject: [PATCH 18/74] Update start-druid-main.py (#14471) Quick typo correction. --- examples/bin/start-druid-main.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/bin/start-druid-main.py b/examples/bin/start-druid-main.py index d7971572307..36d07c17b67 100644 --- a/examples/bin/start-druid-main.py +++ b/examples/bin/start-druid-main.py @@ -154,7 +154,7 @@ sample usage: '\'common.jvm.config\' & \'common.runtime.properties\' files. \n' 'If this argument is not given, config from \n' 'conf/druid/auto directory is used.\n' - 'Note. zookeeper config cannot be overriden.\n') + 'Note. zookeeper config cannot be overridden.\n') parser.add_argument('--compute', action='store_true', help='Does not start Druid, only displays the memory allocated \n' 'to each service if started with the given total memory.\n') From 155fde33ff7cccdb7dfd9ece3b3fd8f800f5948f Mon Sep 17 00:00:00 2001 From: Rishabh Singh <6513075+findingrish@users.noreply.github.com> Date: Fri, 23 Jun 2023 16:51:08 +0530 Subject: [PATCH 19/74] Add metrics to SegmentMetadataCache refresh (#14453) New metrics: - `segment/metadatacache/refresh/time`: time taken to refresh segments per datasource - `segment/metadatacache/refresh/count`: number of segments being refreshed per datasource --- docs/operations/metrics.md | 3 ++ .../calcite/schema/SegmentMetadataCache.java | 16 ++++++- .../schema/SegmentMetadataCacheTest.java | 48 +++++++++++++++++++ 3 files changed, 65 insertions(+), 2 deletions(-) diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 16aaca90837..f85cbec77df 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -66,6 +66,9 @@ Metrics may have additional dimensions beyond those listed above. |`sqlQuery/bytes`|Number of bytes returned in the SQL query response.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`, `engine`| | |`init/serverview/time`|Time taken to initialize the broker server view. Useful to detect if brokers are taking too long to start.||Depends on the number of segments.| |`init/metadatacache/time`|Time taken to initialize the broker segment metadata cache. Useful to detect if brokers are taking too long to start||Depends on the number of segments.| +|`segment/metadatacache/refresh/count`|Number of segments to refresh in broker segment metadata cache.|`dataSource`| +|`segment/metadatacache/refresh/time`|Time taken to refresh segments in broker segment metadata cache.|`dataSource`| + ### Historical diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java index 71c19734ec0..ea3dc395671 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicates; +import com.google.common.base.Stopwatch; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Interner; @@ -48,6 +49,7 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.TableDataSource; @@ -88,6 +90,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -707,6 +710,8 @@ public class SegmentMetadataCache private Set refreshSegmentsForDataSource(final String dataSource, final Set segments) throws IOException { + final Stopwatch stopwatch = Stopwatch.createStarted(); + if (!segments.stream().allMatch(segmentId -> segmentId.getDataSource().equals(dataSource))) { // Sanity check. We definitely expect this to pass. throw new ISE("'segments' must all match 'dataSource'!"); @@ -714,7 +719,10 @@ public class SegmentMetadataCache log.debug("Refreshing metadata for dataSource[%s].", dataSource); - final long startTime = System.currentTimeMillis(); + final ServiceMetricEvent.Builder builder = + new ServiceMetricEvent.Builder().setDimension(DruidMetrics.DATASOURCE, dataSource); + + emitter.emit(builder.build("segment/metadatacache/refresh/count", segments.size())); // Segment id string -> SegmentId object. final Map segmentIdMap = Maps.uniqueIndex(segments, SegmentId::toString); @@ -783,10 +791,14 @@ public class SegmentMetadataCache yielder.close(); } + long refreshDurationMillis = stopwatch.elapsed(TimeUnit.MILLISECONDS); + + emitter.emit(builder.build("segment/metadatacache/refresh/time", refreshDurationMillis)); + log.debug( "Refreshed metadata for dataSource [%s] in %,d ms (%d segments queried, %d segments left).", dataSource, - System.currentTimeMillis() - startTime, + refreshDurationMillis, retVal.size(), segments.size() - retVal.size() ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java index fb55d227109..0414878b4a5 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java @@ -39,6 +39,8 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.TableDataSource; @@ -1447,6 +1449,52 @@ public class SegmentMetadataCacheTest extends SegmentMetadataCacheCommon Assert.assertNull(schema.getDatasource("wat")); } + @Test + public void testRefreshShouldEmitMetrics() throws InterruptedException, IOException + { + String datasource = "xyz"; + CountDownLatch addSegmentLatch = new CountDownLatch(2); + StubServiceEmitter emitter = new StubServiceEmitter("broker", "host"); + SegmentMetadataCache schema = new SegmentMetadataCache( + CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + serverView, + segmentManager, + new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new BrokerInternalQueryConfig(), + emitter + ) + { + @Override + protected void addSegment(final DruidServerMetadata server, final DataSegment segment) + { + super.addSegment(server, segment); + if (datasource.equals(segment.getDataSource())) { + addSegmentLatch.countDown(); + } + } + + @Override + void removeSegment(final DataSegment segment) + { + super.removeSegment(segment); + } + }; + + List segments = ImmutableList.of( + newSegment(datasource, 1), + newSegment(datasource, 2) + ); + serverView.addSegment(segments.get(0), ServerType.HISTORICAL); + serverView.addSegment(segments.get(1), ServerType.REALTIME); + Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); + schema.refresh(segments.stream().map(DataSegment::getId).collect(Collectors.toSet()), Sets.newHashSet(datasource)); + + emitter.verifyEmitted("segment/metadatacache/refresh/time", ImmutableMap.of(DruidMetrics.DATASOURCE, datasource), 1); + emitter.verifyEmitted("segment/metadatacache/refresh/count", ImmutableMap.of(DruidMetrics.DATASOURCE, datasource), 1); + } + private static DataSegment newSegment(String datasource, int partitionId) { return new DataSegment( From ddd0fc1b855b2d9645d052561a1cf4e89544ed8a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 23 Jun 2023 15:23:59 -0700 Subject: [PATCH 20/74] S3: Attach SSE key to doesObjectExist calls. (#14290) * S3: Attach SSE key to doesObjectExist calls. We did not previously attach the SSE key to the doesObjectExist request, leading to an inconsistency that may cause problems on "S3-compatible" implementations. This patch implements doesObjectExist using similar logic to the S3 client itself, but calls our implementation of getObjectMetadata rather than the S3 client's, ensuring the request is decorated with the SSE key. * Fix tests. --- .../s3/ServerSideEncryptingAmazonS3.java | 30 ++-- .../s3/output/S3StorageConnectorTest.java | 158 +++++++++++------- 2 files changed, 115 insertions(+), 73 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java index af733dde247..320a0b9a6f9 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java @@ -25,6 +25,7 @@ import com.amazonaws.services.s3.AmazonS3Client; import com.amazonaws.services.s3.AmazonS3ClientBuilder; import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; import com.amazonaws.services.s3.model.AccessControlList; +import com.amazonaws.services.s3.model.AmazonS3Exception; import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; import com.amazonaws.services.s3.model.CopyObjectRequest; @@ -43,11 +44,8 @@ import com.amazonaws.services.s3.model.S3Object; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import java.io.ByteArrayInputStream; import java.io.File; -import java.io.InputStream; /** * {@link AmazonS3} wrapper with {@link ServerSideEncryption}. Every {@link AmazonS3#putObject}, @@ -76,7 +74,20 @@ public class ServerSideEncryptingAmazonS3 public boolean doesObjectExist(String bucket, String objectName) { - return amazonS3.doesObjectExist(bucket, objectName); + try { + // Ignore return value, just want to see if we can get the metadata at all. + getObjectMetadata(bucket, objectName); + return true; + } + catch (AmazonS3Exception e) { + if (e.getStatusCode() == 404) { + // Object not found. + return false; + } else { + // Some other error: re-throw. + throw e; + } + } } public ListObjectsV2Result listObjectsV2(ListObjectsV2Request request) @@ -107,22 +118,11 @@ public class ServerSideEncryptingAmazonS3 return amazonS3.getObject(serverSideEncryption.decorate(request)); } - public PutObjectResult putObject(String bucket, String key, String content) - { - final InputStream in = new ByteArrayInputStream(StringUtils.toUtf8(content)); - return putObject(new PutObjectRequest(bucket, key, in, new ObjectMetadata())); - } - public PutObjectResult putObject(String bucket, String key, File file) { return putObject(new PutObjectRequest(bucket, key, file)); } - public PutObjectResult putObject(String bucket, String key, InputStream in, ObjectMetadata objectMetadata) - { - return putObject(new PutObjectRequest(bucket, key, in, objectMetadata)); - } - public PutObjectResult putObject(PutObjectRequest request) { return amazonS3.putObject(serverSideEncryption.decorate(request)); diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java index a1d42508119..380c5cb1e50 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java @@ -20,7 +20,9 @@ package org.apache.druid.storage.s3.output; import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.model.AmazonS3Exception; import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.ListObjectsV2Request; import com.amazonaws.services.s3.model.ListObjectsV2Result; @@ -34,10 +36,13 @@ import org.apache.druid.storage.s3.NoopServerSideEncryption; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; import org.easymock.Capture; import org.easymock.EasyMock; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.internal.matchers.ThrowableCauseMatcher; import org.junit.rules.TemporaryFolder; import java.io.BufferedReader; @@ -52,22 +57,20 @@ import java.util.stream.Collectors; public class S3StorageConnectorTest { - private static final AmazonS3Client S3_CLIENT = EasyMock.createMock(AmazonS3Client.class); - private static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3( - S3_CLIENT, - new NoopServerSideEncryption() - ); - private static final String BUCKET = "BUCKET"; private static final String PREFIX = "P/R/E/F/I/X"; public static final String TEST_FILE = "test.csv"; + private final AmazonS3Client s3Client = EasyMock.createMock(AmazonS3Client.class); + private final ServerSideEncryptingAmazonS3 service = new ServerSideEncryptingAmazonS3( + s3Client, + new NoopServerSideEncryption() + ); + private final ListObjectsV2Result testResult = EasyMock.createMock(ListObjectsV2Result.class); + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - public static final ListObjectsV2Result TEST_RESULT = EasyMock.createMock(ListObjectsV2Result.class); - - private StorageConnector storageConnector; @Before @@ -81,40 +84,79 @@ public class S3StorageConnectorTest null, null, true - ), SERVICE); + ), service); } catch (IOException e) { throw new RuntimeException(e); } } - @Test - public void pathExists() throws IOException + public void pathExists_yes() throws IOException { - EasyMock.reset(S3_CLIENT); - EasyMock.expect(S3_CLIENT.doesObjectExist(BUCKET, PREFIX + "/" + TEST_FILE)).andReturn(true); - EasyMock.replay(S3_CLIENT); + final Capture request = Capture.newInstance(); + EasyMock.reset(s3Client); + EasyMock.expect(s3Client.getObjectMetadata(EasyMock.capture(request))) + .andReturn(new ObjectMetadata()); + EasyMock.replay(s3Client); Assert.assertTrue(storageConnector.pathExists(TEST_FILE)); - EasyMock.reset(S3_CLIENT); - Assert.assertFalse(storageConnector.pathExists("test1.csv")); + Assert.assertEquals(BUCKET, request.getValue().getBucketName()); + Assert.assertEquals(PREFIX + "/" + TEST_FILE, request.getValue().getKey()); + EasyMock.verify(s3Client); } + @Test + public void pathExists_notFound() throws IOException + { + final Capture request = Capture.newInstance(); + final AmazonS3Exception e = new AmazonS3Exception("not found"); + e.setStatusCode(404); + + EasyMock.reset(s3Client); + EasyMock.expect(s3Client.getObjectMetadata(EasyMock.capture(request))) + .andThrow(e); + EasyMock.replay(s3Client); + Assert.assertFalse(storageConnector.pathExists(TEST_FILE)); + Assert.assertEquals(BUCKET, request.getValue().getBucketName()); + Assert.assertEquals(PREFIX + "/" + TEST_FILE, request.getValue().getKey()); + EasyMock.verify(s3Client); + } + + @Test + public void pathExists_error() + { + final Capture request = Capture.newInstance(); + final AmazonS3Exception e = new AmazonS3Exception("not found"); + e.setStatusCode(403); + + EasyMock.reset(s3Client); + EasyMock.expect(s3Client.getObjectMetadata(EasyMock.capture(request))) + .andThrow(e); + EasyMock.replay(s3Client); + final IOException e2 = Assert.assertThrows( + IOException.class, + () -> storageConnector.pathExists(TEST_FILE) + ); + Assert.assertEquals(BUCKET, request.getValue().getBucketName()); + Assert.assertEquals(PREFIX + "/" + TEST_FILE, request.getValue().getKey()); + MatcherAssert.assertThat(e2, ThrowableCauseMatcher.hasCause(CoreMatchers.instanceOf(AmazonS3Exception.class))); + EasyMock.verify(s3Client); + } @Test public void pathRead() throws IOException { - EasyMock.reset(S3_CLIENT); + EasyMock.reset(s3Client); ObjectMetadata objectMetadata = new ObjectMetadata(); long contentLength = "test".getBytes(StandardCharsets.UTF_8).length; objectMetadata.setContentLength(contentLength); S3Object s3Object = new S3Object(); s3Object.setObjectContent(new ByteArrayInputStream("test".getBytes(StandardCharsets.UTF_8))); - EasyMock.expect(S3_CLIENT.getObjectMetadata(EasyMock.anyObject())).andReturn(objectMetadata); - EasyMock.expect(S3_CLIENT.getObject( + EasyMock.expect(s3Client.getObjectMetadata(EasyMock.anyObject())).andReturn(objectMetadata); + EasyMock.expect(s3Client.getObject( new GetObjectRequest(BUCKET, PREFIX + "/" + TEST_FILE).withRange(0, contentLength - 1)) ).andReturn(s3Object); - EasyMock.replay(S3_CLIENT); + EasyMock.replay(s3Client); String readText = new BufferedReader( new InputStreamReader(storageConnector.read(TEST_FILE), StandardCharsets.UTF_8)) @@ -122,7 +164,7 @@ public class S3StorageConnectorTest .collect(Collectors.joining("\n")); Assert.assertEquals("test", readText); - EasyMock.reset(S3_CLIENT); + EasyMock.reset(s3Client); } @Test @@ -134,123 +176,124 @@ public class S3StorageConnectorTest for (int start = 0; start < data.length(); start++) { for (int length = 1; length <= data.length() - start; length++) { String dataQueried = data.substring(start, start + length); - EasyMock.reset(S3_CLIENT); + EasyMock.reset(s3Client); S3Object s3Object = new S3Object(); s3Object.setObjectContent( new ByteArrayInputStream(dataQueried.getBytes(StandardCharsets.UTF_8)) ); EasyMock.expect( - S3_CLIENT.getObject( + s3Client.getObject( new GetObjectRequest(BUCKET, PREFIX + "/" + TEST_FILE).withRange(start, start + length - 1) ) ).andReturn(s3Object); - EasyMock.replay(S3_CLIENT); + EasyMock.replay(s3Client); InputStream is = storageConnector.readRange(TEST_FILE, start, length); byte[] dataBytes = new byte[length]; Assert.assertEquals(length, is.read(dataBytes)); Assert.assertEquals(-1, is.read()); // reading further produces no data Assert.assertEquals(dataQueried, new String(dataBytes, StandardCharsets.UTF_8)); - EasyMock.reset(S3_CLIENT); + EasyMock.reset(s3Client); } } // empty read - EasyMock.reset(S3_CLIENT); + EasyMock.reset(s3Client); S3Object s3Object = new S3Object(); s3Object.setObjectContent( new ByteArrayInputStream("".getBytes(StandardCharsets.UTF_8)) ); EasyMock.expect( - S3_CLIENT.getObject( + s3Client.getObject( new GetObjectRequest(BUCKET, PREFIX + "/" + TEST_FILE).withRange(0, -1) ) ).andReturn(s3Object); - EasyMock.replay(S3_CLIENT); + EasyMock.replay(s3Client); InputStream is = storageConnector.readRange(TEST_FILE, 0, 0); byte[] dataBytes = new byte[0]; Assert.assertEquals(is.read(dataBytes), -1); Assert.assertEquals("", new String(dataBytes, StandardCharsets.UTF_8)); - EasyMock.reset(S3_CLIENT); + EasyMock.reset(s3Client); } @Test public void testDeleteSinglePath() throws IOException { - EasyMock.reset(S3_CLIENT); - S3_CLIENT.deleteObject(BUCKET, PREFIX + "/" + TEST_FILE); + EasyMock.reset(s3Client); + s3Client.deleteObject(BUCKET, PREFIX + "/" + TEST_FILE); EasyMock.expectLastCall(); storageConnector.deleteFile(TEST_FILE); - EasyMock.reset(S3_CLIENT); + EasyMock.reset(s3Client); } - @Test public void testDeleteMultiplePaths() throws IOException { - EasyMock.reset(S3_CLIENT); + EasyMock.reset(s3Client); String testFile2 = "file2"; DeleteObjectsRequest deleteObjectsRequest = new DeleteObjectsRequest(BUCKET); deleteObjectsRequest.withKeys(PREFIX + "/" + TEST_FILE, PREFIX + "/" + testFile2); Capture capturedArgument = EasyMock.newCapture(); - EasyMock.expect(S3_CLIENT.deleteObjects(EasyMock.capture(capturedArgument))).andReturn(null).once(); - EasyMock.replay(S3_CLIENT); + EasyMock.expect(s3Client.deleteObjects(EasyMock.capture(capturedArgument))).andReturn(null).once(); + EasyMock.replay(s3Client); storageConnector.deleteFiles(Lists.newArrayList(TEST_FILE, testFile2)); - Assert.assertEquals(convertDeleteObjectsRequestToString(deleteObjectsRequest), convertDeleteObjectsRequestToString(capturedArgument.getValue())); - EasyMock.reset(S3_CLIENT); + Assert.assertEquals( + convertDeleteObjectsRequestToString(deleteObjectsRequest), + convertDeleteObjectsRequestToString(capturedArgument.getValue()) + ); + EasyMock.reset(s3Client); } - @Test public void testPathDeleteRecursively() throws IOException { - EasyMock.reset(S3_CLIENT, TEST_RESULT); + EasyMock.reset(s3Client, testResult); S3ObjectSummary s3ObjectSummary = new S3ObjectSummary(); s3ObjectSummary.setBucketName(BUCKET); s3ObjectSummary.setKey(PREFIX + "/test/" + TEST_FILE); s3ObjectSummary.setSize(1); - EasyMock.expect(S3_CLIENT.listObjectsV2((ListObjectsV2Request) EasyMock.anyObject())) - .andReturn(TEST_RESULT); + EasyMock.expect(s3Client.listObjectsV2((ListObjectsV2Request) EasyMock.anyObject())) + .andReturn(testResult); - EasyMock.expect(TEST_RESULT.getBucketName()).andReturn("123").anyTimes(); - EasyMock.expect(TEST_RESULT.getObjectSummaries()).andReturn(Collections.singletonList(s3ObjectSummary)).anyTimes(); - EasyMock.expect(TEST_RESULT.isTruncated()).andReturn(false).times(1); - EasyMock.expect(TEST_RESULT.getNextContinuationToken()).andReturn(null); + EasyMock.expect(testResult.getBucketName()).andReturn("123").anyTimes(); + EasyMock.expect(testResult.getObjectSummaries()).andReturn(Collections.singletonList(s3ObjectSummary)).anyTimes(); + EasyMock.expect(testResult.isTruncated()).andReturn(false).times(1); + EasyMock.expect(testResult.getNextContinuationToken()).andReturn(null); Capture capturedArgument = EasyMock.newCapture(); - EasyMock.expect(S3_CLIENT.deleteObjects(EasyMock.and( + EasyMock.expect(s3Client.deleteObjects(EasyMock.and( EasyMock.capture(capturedArgument), EasyMock.isA(DeleteObjectsRequest.class) ))).andReturn(null); - EasyMock.replay(S3_CLIENT, TEST_RESULT); + EasyMock.replay(s3Client, testResult); storageConnector.deleteRecursively("test"); Assert.assertEquals(1, capturedArgument.getValue().getKeys().size()); Assert.assertEquals(PREFIX + "/test/" + TEST_FILE, capturedArgument.getValue().getKeys().get(0).getKey()); - EasyMock.reset(S3_CLIENT, TEST_RESULT); + EasyMock.reset(s3Client, testResult); } @Test public void testListDir() throws IOException { - EasyMock.reset(S3_CLIENT, TEST_RESULT); + EasyMock.reset(s3Client, testResult); S3ObjectSummary s3ObjectSummary = new S3ObjectSummary(); s3ObjectSummary.setBucketName(BUCKET); s3ObjectSummary.setKey(PREFIX + "/test/" + TEST_FILE); s3ObjectSummary.setSize(1); - EasyMock.expect(TEST_RESULT.getObjectSummaries()).andReturn(Collections.singletonList(s3ObjectSummary)).times(2); - EasyMock.expect(TEST_RESULT.isTruncated()).andReturn(false); - EasyMock.expect(TEST_RESULT.getNextContinuationToken()).andReturn(null); - EasyMock.expect(S3_CLIENT.listObjectsV2((ListObjectsV2Request) EasyMock.anyObject())) - .andReturn(TEST_RESULT); - EasyMock.replay(S3_CLIENT, TEST_RESULT); + EasyMock.expect(testResult.getObjectSummaries()).andReturn(Collections.singletonList(s3ObjectSummary)).times(2); + EasyMock.expect(testResult.isTruncated()).andReturn(false); + EasyMock.expect(testResult.getNextContinuationToken()).andReturn(null); + EasyMock.expect(s3Client.listObjectsV2((ListObjectsV2Request) EasyMock.anyObject())) + .andReturn(testResult); + EasyMock.replay(s3Client, testResult); List listDirResult = Lists.newArrayList(storageConnector.listDir("test/")); Assert.assertEquals(ImmutableList.of(TEST_FILE), listDirResult); @@ -264,5 +307,4 @@ public class S3StorageConnectorTest .collect( Collectors.joining()); } - } From 1d6c9657ec362199b80993b68a0bfbe4118ea29a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 23 Jun 2023 15:24:15 -0700 Subject: [PATCH 21/74] Clarify compaction docs. (#14225) * Clarify compaction docs. The prior wording made it sound like segmentGranularity, queryGranularity, and rollup are always required for granularitySpec. They are not required, but they are strongly recommended. The adjusted wording hopefully does a better job of making that clear. * Fix link. * Wording adjustments. --- docs/data-management/automatic-compaction.md | 8 ++++---- docs/data-management/compaction.md | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/data-management/automatic-compaction.md b/docs/data-management/automatic-compaction.md index 866ca2407dd..d7c0bb84957 100644 --- a/docs/data-management/automatic-compaction.md +++ b/docs/data-management/automatic-compaction.md @@ -115,11 +115,11 @@ The following properties are automatically set by the Coordinator: * `id`: Generated using the task type, datasource name, interval, and timestamp. The task ID is prefixed with `coordinator-issued`. * `context`: Set according to the user-provided `taskContext`. -Compaction tasks fetch all [relevant segments](compaction.md#compaction-io-configuration) prior to launching any subtasks, -_unless_ the following items are all set. It is strongly recommended to set all of these items to maximize performance -and minimize disk usage of the `compact` tasks launched by auto-compaction: +Compaction tasks typically fetch all [relevant segments](compaction.md#compaction-io-configuration) prior to launching any subtasks, +_unless_ the following properties are all set to non-null values. It is strongly recommended to set them to non-null values to +maximize performance and minimize disk usage of the `compact` tasks launched by auto-compaction: -- [`granularitySpec`](compaction.md#compaction-granularity-spec). All three values must be set to non-null values: `segmentGranularity`, `queryGranularity`, and `rollup`. +- [`granularitySpec`](compaction.md#compaction-granularity-spec), with non-null values for each of `segmentGranularity`, `queryGranularity`, and `rollup` - [`dimensionsSpec`](compaction.md#compaction-dimensions-spec) - `metricsSpec` diff --git a/docs/data-management/compaction.md b/docs/data-management/compaction.md index 3e833469e8e..96a7c076092 100644 --- a/docs/data-management/compaction.md +++ b/docs/data-management/compaction.md @@ -138,9 +138,9 @@ To control the number of result segments per time chunk, you can set [`maxRowsPe A compaction task internally generates an `index` or `index_parallel` task spec for performing compaction work with some fixed parameters. For example, its `inputSource` is always the [`druid` input source](../ingestion/input-sources.md), and `dimensionsSpec` and `metricsSpec` include all dimensions and metrics of the input segments by default. -Compaction tasks fetch all [relevant segments](#compaction-io-configuration) prior to launching any subtasks, _unless_ the following items are all set. It is strongly recommended to set all of these items to maximize performance and minimize disk usage of the `compact` task: +Compaction tasks typically fetch all [relevant segments](#compaction-io-configuration) prior to launching any subtasks, _unless_ the following properties are all set to non-null values. It is strongly recommended to set them to non-null values to maximize performance and minimize disk usage of the `compact` task: -- [`granularitySpec`](#compaction-granularity-spec). All three values must be set to non-null values: `segmentGranularity`, `queryGranularity`, and `rollup`. +- [`granularitySpec`](#compaction-granularity-spec), with non-null values for each of `segmentGranularity`, `queryGranularity`, and `rollup` - [`dimensionsSpec`](#compaction-dimensions-spec) - `metricsSpec` From 3d19b748fb9fde49f074a8234e797a46bd894d48 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 23 Jun 2023 16:25:04 -0700 Subject: [PATCH 22/74] SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-literal. (#14249) * SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-literal. Four main changes: 1) Provide aggregatorBuilder, a more consistent way of defining the SqlAggFunction we need for all of our SQL aggregators. The mechanism is analogous to the one we already use for SQL functions (OperatorConversions.operatorBuilder). 2) Allow CASTs of constants to be considered as "literalOperands". This fixes an issue where various of our operators are defined with OperandTypes.LITERAL as part of their checkers, which doesn't allow casts. However, in these cases we generally _do_ want to allow casts. The important piece is that the value must be reducible to a constant, not that the SQL text is literally a literal. 3) Update DataSketches SQL aggregators to use the new aggregatorBuilder functionality. The main user-visible effect here is [2]: the aggregators would now accept, for example, "CAST(0.99 AS DOUBLE)" as a literal argument. Other aggregators could be updated in a future patch. 4) Rename "requiredOperands" to "requiredOperandCount", because the old name was confusing. (It rhymes with "literalOperands" but the arguments mean different things.) * Adjust method calls. --- ...ketchApproxCountDistinctSqlAggregator.java | 42 +--- .../HllSketchEstimateOperatorConversion.java | 2 +- ...mateWithErrorBoundsOperatorConversion.java | 2 +- .../hll/sql/HllSketchObjectSqlAggregator.java | 41 +--- ...blesSketchApproxQuantileSqlAggregator.java | 44 +--- .../sql/DoublesSketchObjectSqlAggregator.java | 40 +--- ...ketchApproxCountDistinctSqlAggregator.java | 42 +--- .../sql/ThetaSketchObjectSqlAggregator.java | 40 +--- .../hll/sql/HllSketchSqlAggregatorTest.java | 10 +- .../sql/DoublesSketchSqlAggregatorTest.java | 2 +- .../query/sql/SleepOperatorConversion.java | 2 +- .../tools/SleepOperatorConversion.java | 2 +- .../expression/OperatorConversions.java | 226 +++++++++++++----- .../builtin/BTrimOperatorConversion.java | 2 +- .../builtin/ContainsOperatorConversion.java | 2 +- .../builtin/DateTruncOperatorConversion.java | 2 +- .../builtin/LPadOperatorConversion.java | 2 +- .../builtin/LTrimOperatorConversion.java | 2 +- .../builtin/ParseLongOperatorConversion.java | 2 +- .../builtin/RPadOperatorConversion.java | 2 +- .../builtin/RTrimOperatorConversion.java | 2 +- .../RegexpExtractOperatorConversion.java | 2 +- .../builtin/RegexpLikeOperatorConversion.java | 2 +- .../builtin/RoundOperatorConversion.java | 2 +- .../builtin/SubstringOperatorConversion.java | 2 +- .../builtin/TextcatOperatorConversion.java | 2 +- .../builtin/TimeCeilOperatorConversion.java | 2 +- .../TimeExtractOperatorConversion.java | 2 +- .../builtin/TimeFloorOperatorConversion.java | 2 +- .../builtin/TimeFormatOperatorConversion.java | 2 +- .../builtin/TimeParseOperatorConversion.java | 2 +- .../builtin/TimeShiftOperatorConversion.java | 2 +- .../builtin/TruncateOperatorConversion.java | 2 +- .../expression/OperatorConversionsTest.java | 66 +++-- .../planner/DruidOperatorTableTest.java | 2 +- .../calcite/planner/DruidRexExecutorTest.java | 2 +- .../calcite/schema/InformationSchemaTest.java | 6 +- 37 files changed, 324 insertions(+), 287 deletions(-) diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java index 461b2316f10..29f7c819be1 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java @@ -21,24 +21,30 @@ package org.apache.druid.query.aggregation.datasketches.hll.sql; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.InferTypes; -import org.apache.calcite.sql.type.OperandTypes; -import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; +import org.apache.druid.sql.calcite.expression.OperatorConversions; import java.util.Collections; public class HllSketchApproxCountDistinctSqlAggregator extends HllSketchBaseSqlAggregator implements SqlAggregator { public static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL"; - - private static final SqlAggFunction FUNCTION_INSTANCE = new HllSketchApproxCountDistinctSqlAggFunction(); + private static final SqlAggFunction FUNCTION_INSTANCE = + OperatorConversions.aggregatorBuilder(NAME) + .operandNames("column", "lgK", "tgtHllType") + .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING) + .operandTypeInference(InferTypes.VARCHAR_1024) + .requiredOperandCount(1) + .literalOperands(1, 2) + .returnTypeNonNull(SqlTypeName.BIGINT) + .functionCategory(SqlFunctionCategory.NUMERIC) + .build(); public HllSketchApproxCountDistinctSqlAggregator() { @@ -66,30 +72,4 @@ public class HllSketchApproxCountDistinctSqlAggregator extends HllSketchBaseSqlA ) : null ); } - - private static class HllSketchApproxCountDistinctSqlAggFunction extends SqlAggFunction - { - private static final String SIGNATURE = "'" + NAME + "(column, lgK, tgtHllType)'"; - - HllSketchApproxCountDistinctSqlAggFunction() - { - super( - NAME, - null, - SqlKind.OTHER_FUNCTION, - ReturnTypes.explicit(SqlTypeName.BIGINT), - InferTypes.VARCHAR_1024, - OperandTypes.or( - OperandTypes.ANY, - OperandTypes.and( - OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL, OperandTypes.LITERAL), - OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING) - ) - ), - SqlFunctionCategory.NUMERIC, - false, - false - ); - } - } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateOperatorConversion.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateOperatorConversion.java index fa154933531..dbc4bf8bcd3 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateOperatorConversion.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateOperatorConversion.java @@ -47,7 +47,7 @@ public class HllSketchEstimateOperatorConversion implements SqlOperatorConversio private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME)) .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.BOOLEAN) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeInference(ReturnTypes.DOUBLE) .build(); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateWithErrorBoundsOperatorConversion.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateWithErrorBoundsOperatorConversion.java index 83a0ce43bf5..cc89d4c7a56 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateWithErrorBoundsOperatorConversion.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateWithErrorBoundsOperatorConversion.java @@ -46,7 +46,7 @@ public class HllSketchEstimateWithErrorBoundsOperatorConversion implements SqlOp private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME)) .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.INTEGER) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.OTHER) .build(); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchObjectSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchObjectSqlAggregator.java index 56fdbd33811..8e695148b97 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchObjectSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchObjectSqlAggregator.java @@ -21,22 +21,29 @@ package org.apache.druid.query.aggregation.datasketches.hll.sql; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.InferTypes; -import org.apache.calcite.sql.type.OperandTypes; -import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; +import org.apache.druid.sql.calcite.expression.OperatorConversions; import java.util.Collections; public class HllSketchObjectSqlAggregator extends HllSketchBaseSqlAggregator implements SqlAggregator { - private static final SqlAggFunction FUNCTION_INSTANCE = new HllSketchSqlAggFunction(); private static final String NAME = "DS_HLL"; + private static final SqlAggFunction FUNCTION_INSTANCE = + OperatorConversions.aggregatorBuilder(NAME) + .operandNames("column", "lgK", "tgtHllType") + .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING) + .operandTypeInference(InferTypes.VARCHAR_1024) + .requiredOperandCount(1) + .literalOperands(1, 2) + .returnTypeNonNull(SqlTypeName.OTHER) + .functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION) + .build(); public HllSketchObjectSqlAggregator() { @@ -61,30 +68,4 @@ public class HllSketchObjectSqlAggregator extends HllSketchBaseSqlAggregator imp null ); } - - private static class HllSketchSqlAggFunction extends SqlAggFunction - { - private static final String SIGNATURE = "'" + NAME + "(column, lgK, tgtHllType)'"; - - HllSketchSqlAggFunction() - { - super( - NAME, - null, - SqlKind.OTHER_FUNCTION, - ReturnTypes.explicit(SqlTypeName.OTHER), - InferTypes.VARCHAR_1024, - OperandTypes.or( - OperandTypes.ANY, - OperandTypes.and( - OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL, OperandTypes.LITERAL), - OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING) - ) - ), - SqlFunctionCategory.USER_DEFINED_FUNCTION, - false, - false - ); - } - } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java index 0b4bbd7f11b..7864447c492 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java @@ -28,8 +28,6 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.type.OperandTypes; -import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.java.util.common.StringUtils; @@ -45,19 +43,27 @@ import org.apache.druid.sql.calcite.aggregation.Aggregations; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; +import org.apache.druid.sql.calcite.expression.OperatorConversions; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; import javax.annotation.Nullable; - import java.util.List; public class DoublesSketchApproxQuantileSqlAggregator implements SqlAggregator { public static final String CTX_APPROX_QUANTILE_DS_MAX_STREAM_LENGTH = "approxQuantileDsMaxStreamLength"; - private static final SqlAggFunction FUNCTION_INSTANCE = new DoublesSketchApproxQuantileSqlAggFunction(); private static final String NAME = "APPROX_QUANTILE_DS"; + private static final SqlAggFunction FUNCTION_INSTANCE = + OperatorConversions.aggregatorBuilder(NAME) + .operandNames("column", "probability", "k") + .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.EXACT_NUMERIC) + .returnTypeNonNull(SqlTypeName.DOUBLE) + .requiredOperandCount(2) + .literalOperands(1, 2) + .functionCategory(SqlFunctionCategory.NUMERIC) + .build(); @Override public SqlAggFunction calciteFunction() @@ -212,34 +218,4 @@ public class DoublesSketchApproxQuantileSqlAggregator implements SqlAggregator DoublesSketchAggregatorFactory.DEFAULT_MAX_STREAM_LENGTH ); } - - private static class DoublesSketchApproxQuantileSqlAggFunction extends SqlAggFunction - { - private static final String SIGNATURE1 = "'" + NAME + "(column, probability)'"; - private static final String SIGNATURE2 = "'" + NAME + "(column, probability, k)'"; - - DoublesSketchApproxQuantileSqlAggFunction() - { - super( - NAME, - null, - SqlKind.OTHER_FUNCTION, - ReturnTypes.explicit(SqlTypeName.DOUBLE), - null, - OperandTypes.or( - OperandTypes.and( - OperandTypes.sequence(SIGNATURE1, OperandTypes.ANY, OperandTypes.LITERAL), - OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC) - ), - OperandTypes.and( - OperandTypes.sequence(SIGNATURE2, OperandTypes.ANY, OperandTypes.LITERAL, OperandTypes.LITERAL), - OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.EXACT_NUMERIC) - ) - ), - SqlFunctionCategory.NUMERIC, - false, - false - ); - } - } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java index b82f02fc451..049e1284a91 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java @@ -28,8 +28,6 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.type.OperandTypes; -import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.java.util.common.StringUtils; @@ -43,17 +41,25 @@ import org.apache.druid.sql.calcite.aggregation.Aggregations; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; +import org.apache.druid.sql.calcite.expression.OperatorConversions; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; import javax.annotation.Nullable; - import java.util.List; public class DoublesSketchObjectSqlAggregator implements SqlAggregator { - private static final SqlAggFunction FUNCTION_INSTANCE = new DoublesSketchSqlAggFunction(); private static final String NAME = "DS_QUANTILES_SKETCH"; + private static final SqlAggFunction FUNCTION_INSTANCE = + OperatorConversions.aggregatorBuilder(NAME) + .operandNames("column", "k") + .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.EXACT_NUMERIC) + .returnTypeNonNull(SqlTypeName.OTHER) + .requiredOperandCount(1) + .literalOperands(1) + .functionCategory(SqlFunctionCategory.NUMERIC) + .build(); @Override public SqlAggFunction calciteFunction() @@ -139,30 +145,4 @@ public class DoublesSketchObjectSqlAggregator implements SqlAggregator null ); } - - private static class DoublesSketchSqlAggFunction extends SqlAggFunction - { - private static final String SIGNATURE2 = "'" + NAME + "(column, k)'"; - - DoublesSketchSqlAggFunction() - { - super( - NAME, - null, - SqlKind.OTHER_FUNCTION, - ReturnTypes.explicit(SqlTypeName.OTHER), - null, - OperandTypes.or( - OperandTypes.ANY, - OperandTypes.and( - OperandTypes.sequence(SIGNATURE2, OperandTypes.ANY, OperandTypes.LITERAL), - OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.EXACT_NUMERIC) - ) - ), - SqlFunctionCategory.USER_DEFINED_FUNCTION, - false, - false - ); - } - } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchApproxCountDistinctSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchApproxCountDistinctSqlAggregator.java index 70eb943a8c7..eac77901f1d 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchApproxCountDistinctSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchApproxCountDistinctSqlAggregator.java @@ -21,24 +21,30 @@ package org.apache.druid.query.aggregation.datasketches.theta.sql; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.InferTypes; -import org.apache.calcite.sql.type.OperandTypes; -import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; +import org.apache.druid.sql.calcite.expression.OperatorConversions; import java.util.Collections; public class ThetaSketchApproxCountDistinctSqlAggregator extends ThetaSketchBaseSqlAggregator implements SqlAggregator { public static final String NAME = "APPROX_COUNT_DISTINCT_DS_THETA"; - - private static final SqlAggFunction FUNCTION_INSTANCE = new ThetaSketchSqlAggFunction(); + private static final SqlAggFunction FUNCTION_INSTANCE = + OperatorConversions.aggregatorBuilder(NAME) + .operandNames("column", "size") + .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC) + .operandTypeInference(InferTypes.VARCHAR_1024) + .requiredOperandCount(1) + .literalOperands(1) + .returnTypeNonNull(SqlTypeName.BIGINT) + .functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION) + .build(); public ThetaSketchApproxCountDistinctSqlAggregator() { @@ -66,30 +72,4 @@ public class ThetaSketchApproxCountDistinctSqlAggregator extends ThetaSketchBase ) : null ); } - - private static class ThetaSketchSqlAggFunction extends SqlAggFunction - { - private static final String SIGNATURE = "'" + NAME + "(column, size)'"; - - ThetaSketchSqlAggFunction() - { - super( - NAME, - null, - SqlKind.OTHER_FUNCTION, - ReturnTypes.explicit(SqlTypeName.BIGINT), - InferTypes.VARCHAR_1024, - OperandTypes.or( - OperandTypes.ANY, - OperandTypes.and( - OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL), - OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC) - ) - ), - SqlFunctionCategory.NUMERIC, - false, - false - ); - } - } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchObjectSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchObjectSqlAggregator.java index 91dd36c2fb0..ac9cefd5f9b 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchObjectSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchObjectSqlAggregator.java @@ -21,20 +21,28 @@ package org.apache.druid.query.aggregation.datasketches.theta.sql; import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.InferTypes; -import org.apache.calcite.sql.type.OperandTypes; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; +import org.apache.druid.sql.calcite.expression.OperatorConversions; import java.util.Collections; public class ThetaSketchObjectSqlAggregator extends ThetaSketchBaseSqlAggregator implements SqlAggregator { - private static final SqlAggFunction FUNCTION_INSTANCE = new ThetaSketchObjectSqlAggFunction(); private static final String NAME = "DS_THETA"; + private static final SqlAggFunction FUNCTION_INSTANCE = + OperatorConversions.aggregatorBuilder(NAME) + .operandNames("column", "size") + .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC) + .operandTypeInference(InferTypes.VARCHAR_1024) + .requiredOperandCount(1) + .literalOperands(1) + .returnTypeInference(ThetaSketchSqlOperators.RETURN_TYPE_INFERENCE) + .functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION) + .build(); public ThetaSketchObjectSqlAggregator() { @@ -59,30 +67,4 @@ public class ThetaSketchObjectSqlAggregator extends ThetaSketchBaseSqlAggregator null ); } - - private static class ThetaSketchObjectSqlAggFunction extends SqlAggFunction - { - private static final String SIGNATURE = "'" + NAME + "(column, size)'"; - - ThetaSketchObjectSqlAggFunction() - { - super( - NAME, - null, - SqlKind.OTHER_FUNCTION, - ThetaSketchSqlOperators.RETURN_TYPE_INFERENCE, - InferTypes.VARCHAR_1024, - OperandTypes.or( - OperandTypes.ANY, - OperandTypes.and( - OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL), - OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC) - ) - ), - SqlFunctionCategory.USER_DEFINED_FUNCTION, - false, - false - ); - } - } } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java index 9d7c4b8c03a..99c93a1cedd 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java @@ -177,18 +177,19 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest + " APPROX_COUNT_DISTINCT(SUBSTRING(dim2, 1, 1)),\n" // on extractionFn, using generic A.C.D. + " COUNT(DISTINCT SUBSTRING(dim2, 1, 1) || 'x'),\n" // on expression, using COUNT DISTINCT + " APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1, 21, 'HLL_8'),\n" // on native HllSketch column - + " APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1)\n" // on native HllSketch column + + " APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1),\n" // on native HllSketch column + + " APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1, CAST(21 AS BIGINT))\n" // also native column + "FROM druid.foo"; final List expectedResults; if (NullHandling.replaceWithDefault()) { expectedResults = ImmutableList.of( - new Object[]{6L, 2L, 2L, 1L, 2L, 5L, 5L} + new Object[]{6L, 2L, 2L, 1L, 2L, 5L, 5L, 5L} ); } else { expectedResults = ImmutableList.of( - new Object[]{6L, 2L, 2L, 1L, 1L, 5L, 5L} + new Object[]{6L, 2L, 2L, 1L, 1L, 5L, 5L, 5L} ); } @@ -252,7 +253,8 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest ROUND ), new HllSketchMergeAggregatorFactory("a5", "hllsketch_dim1", 21, "HLL_8", null, ROUND), - new HllSketchMergeAggregatorFactory("a6", "hllsketch_dim1", null, null, null, ROUND) + new HllSketchMergeAggregatorFactory("a6", "hllsketch_dim1", null, null, null, ROUND), + new HllSketchMergeAggregatorFactory("a7", "hllsketch_dim1", 21, "HLL_4", null, ROUND) ) ) .context(QUERY_CONTEXT_DEFAULT) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java index 8f244e5302d..184aba375a0 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java @@ -207,7 +207,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest + "APPROX_QUANTILE_DS(qsketch_m1, 0.01),\n" + "APPROX_QUANTILE_DS(qsketch_m1, 0.5, 64),\n" + "APPROX_QUANTILE_DS(qsketch_m1, 0.98, 256),\n" - + "APPROX_QUANTILE_DS(qsketch_m1, 0.99),\n" + + "APPROX_QUANTILE_DS(qsketch_m1, CAST(0.99 AS DOUBLE)),\n" + "APPROX_QUANTILE_DS(qsketch_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n" + "APPROX_QUANTILE_DS(qsketch_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n" + "APPROX_QUANTILE_DS(qsketch_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n" diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/query/sql/SleepOperatorConversion.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/sql/SleepOperatorConversion.java index 06d841af3ff..d64947d535c 100644 --- a/extensions-core/testing-tools/src/main/java/org/apache/druid/query/sql/SleepOperatorConversion.java +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/sql/SleepOperatorConversion.java @@ -42,7 +42,7 @@ public class SleepOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("SLEEP") .operandTypes(SqlTypeFamily.NUMERIC) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNullable(SqlTypeName.VARCHAR) // always null .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/SleepOperatorConversion.java b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/SleepOperatorConversion.java index cae087cf71e..abc31d3a434 100644 --- a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/SleepOperatorConversion.java +++ b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/SleepOperatorConversion.java @@ -42,7 +42,7 @@ public class SleepOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("SLEEP") .operandTypes(SqlTypeFamily.NUMERIC) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNullable(SqlTypeName.VARCHAR) // always null .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java index c0efb140174..6e3543d7372 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java @@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; import org.apache.calcite.runtime.CalciteException; +import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlCallBinding; import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlFunctionCategory; @@ -49,6 +50,7 @@ import org.apache.calcite.sql.type.SqlReturnTypeInference; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeTransforms; +import org.apache.calcite.util.Optionality; import org.apache.calcite.util.Static; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -57,12 +59,14 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.ExpressionPostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.DruidTypeSystem; import org.apache.druid.sql.calcite.planner.PlannerContext; import javax.annotation.Nullable; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.function.Function; import java.util.stream.IntStream; @@ -316,23 +320,33 @@ public class OperatorConversions * Returns a builder that helps {@link SqlOperatorConversion} implementations create the {@link SqlFunction} * objects they need to return from {@link SqlOperatorConversion#calciteOperator()}. */ - public static OperatorBuilder operatorBuilder(final String name) + public static OperatorBuilder operatorBuilder(final String name) { - return new OperatorBuilder(name); + return new OperatorBuilder<>(name); } - public static class OperatorBuilder + /** + * Returns a builder that helps {@link SqlAggregator} implementations create the {@link SqlAggFunction} objects + * they need to return from {@link SqlAggregator#calciteFunction()}. + */ + public static OperatorBuilder aggregatorBuilder(final String name) { - private final String name; - private SqlKind kind = SqlKind.OTHER_FUNCTION; - private SqlReturnTypeInference returnTypeInference; - private SqlFunctionCategory functionCategory = SqlFunctionCategory.USER_DEFINED_FUNCTION; + return new AggregatorBuilder(name); + } + + public static class OperatorBuilder + { + protected final String name; + protected SqlKind kind = SqlKind.OTHER_FUNCTION; + protected SqlReturnTypeInference returnTypeInference; + protected SqlFunctionCategory functionCategory = SqlFunctionCategory.USER_DEFINED_FUNCTION; // For operand type checking private SqlOperandTypeChecker operandTypeChecker; + private List operandNames = Collections.emptyList(); private List operandTypes; - private Integer requiredOperands = null; - private int[] literalOperands = null; + private Integer requiredOperandCount; + private int[] literalOperands; private SqlOperandTypeInference operandTypeInference; private OperatorBuilder(final String name) @@ -348,7 +362,7 @@ public class OperatorConversions * {@link #returnTypeNullableArrayWithNullableElements}, or {@link #returnTypeInference(SqlReturnTypeInference)} * must be used before calling {@link #build()}. These methods cannot be mixed; you must call exactly one. */ - public OperatorBuilder returnTypeNonNull(final SqlTypeName typeName) + public OperatorBuilder returnTypeNonNull(final SqlTypeName typeName) { Preconditions.checkState(this.returnTypeInference == null, "Cannot set return type multiple times"); @@ -365,7 +379,7 @@ public class OperatorConversions * {@link #returnTypeNullableArrayWithNullableElements}, or {@link #returnTypeInference(SqlReturnTypeInference)} * must be used before calling {@link #build()}. These methods cannot be mixed; you must call exactly one. */ - public OperatorBuilder returnTypeNullable(final SqlTypeName typeName) + public OperatorBuilder returnTypeNullable(final SqlTypeName typeName) { Preconditions.checkState(this.returnTypeInference == null, "Cannot set return type multiple times"); @@ -382,7 +396,7 @@ public class OperatorConversions * {@link #returnTypeNullableArrayWithNullableElements}, or {@link #returnTypeInference(SqlReturnTypeInference)} * must be used before calling {@link #build()}. These methods cannot be mixed; you must call exactly one. */ - public OperatorBuilder returnTypeCascadeNullable(final SqlTypeName typeName) + public OperatorBuilder returnTypeCascadeNullable(final SqlTypeName typeName) { Preconditions.checkState(this.returnTypeInference == null, "Cannot set return type multiple times"); this.returnTypeInference = ReturnTypes.cascade(ReturnTypes.explicit(typeName), SqlTypeTransforms.TO_NULLABLE); @@ -396,7 +410,7 @@ public class OperatorConversions * {@link #returnTypeArrayWithNullableElements}, or {@link #returnTypeInference(SqlReturnTypeInference)} must be * used before calling {@link #build()}. These methods cannot be mixed; you must call exactly one. */ - public OperatorBuilder returnTypeArrayWithNullableElements(final SqlTypeName elementTypeName) + public OperatorBuilder returnTypeArrayWithNullableElements(final SqlTypeName elementTypeName) { Preconditions.checkState(this.returnTypeInference == null, "Cannot set return type multiple times"); @@ -413,7 +427,7 @@ public class OperatorConversions * {@link #returnTypeArrayWithNullableElements}, or {@link #returnTypeInference(SqlReturnTypeInference)} must be * used before calling {@link #build()}. These methods cannot be mixed; you must call exactly one. */ - public OperatorBuilder returnTypeNullableArrayWithNullableElements(final SqlTypeName elementTypeName) + public OperatorBuilder returnTypeNullableArrayWithNullableElements(final SqlTypeName elementTypeName) { this.returnTypeInference = ReturnTypes.cascade( opBinding -> Calcites.createSqlArrayTypeWithNullability( @@ -434,7 +448,7 @@ public class OperatorConversions * {@link #returnTypeNullableArrayWithNullableElements}, or {@link #returnTypeInference(SqlReturnTypeInference)} * must be used before calling {@link #build()}. These methods cannot be mixed; you must call exactly one. */ - public OperatorBuilder returnTypeInference(final SqlReturnTypeInference returnTypeInference) + public OperatorBuilder returnTypeInference(final SqlReturnTypeInference returnTypeInference) { Preconditions.checkState(this.returnTypeInference == null, "Cannot set return type multiple times"); @@ -447,7 +461,7 @@ public class OperatorConversions * * The default, if not provided, is {@link SqlFunctionCategory#USER_DEFINED_FUNCTION}. */ - public OperatorBuilder functionCategory(final SqlFunctionCategory functionCategory) + public OperatorBuilder functionCategory(final SqlFunctionCategory functionCategory) { this.functionCategory = functionCategory; return this; @@ -459,21 +473,32 @@ public class OperatorConversions * One of {@link #operandTypes(SqlTypeFamily...)} or {@link #operandTypeChecker(SqlOperandTypeChecker)} must be used * before calling {@link #build()}. These methods cannot be mixed; you must call exactly one. */ - public OperatorBuilder operandTypeChecker(final SqlOperandTypeChecker operandTypeChecker) + public OperatorBuilder operandTypeChecker(final SqlOperandTypeChecker operandTypeChecker) { this.operandTypeChecker = operandTypeChecker; return this; } + /** + * Signifies that a function accepts operands with the provided names. This is used to implement + * {@link SqlOperandTypeChecker#getAllowedSignatures(SqlOperator, String)}. If not provided, the + * {@link #operandTypes} are used instead. + */ + public OperatorBuilder operandNames(final String... operandNames) + { + this.operandNames = Arrays.asList(operandNames); + return this; + } + /** * Signifies that a function accepts operands of type family given by {@param operandTypes}. * - * May be used in conjunction with {@link #requiredOperands(int)} and {@link #literalOperands(int...)} in order + * May be used in conjunction with {@link #requiredOperandCount(int)} and {@link #literalOperands(int...)} in order * to further refine operand checking logic. * * For deeper control, use {@link #operandTypeChecker(SqlOperandTypeChecker)} instead. */ - public OperatorBuilder operandTypes(final SqlTypeFamily... operandTypes) + public OperatorBuilder operandTypes(final SqlTypeFamily... operandTypes) { this.operandTypes = Arrays.asList(operandTypes); return this; @@ -489,67 +514,97 @@ public class OperatorConversions * Must be used in conjunction with {@link #operandTypes(SqlTypeFamily...)}; this method is not compatible with * {@link #operandTypeChecker(SqlOperandTypeChecker)}. */ - public OperatorBuilder requiredOperands(final int requiredOperands) + public OperatorBuilder requiredOperandCount(final int requiredOperandCount) { - this.requiredOperands = requiredOperands; + this.requiredOperandCount = requiredOperandCount; return this; } + /** + * Alias for {@link #requiredOperandCount(int)}. Deprecated because it means "operand count" rather than + * "specific operands", and therefore the name can cause confusion with {@link #literalOperands(int...)}. The latter + * really does mean "specific operands". + */ + @Deprecated + @SuppressWarnings("unused") // For compatibility with existing extensions + public OperatorBuilder requiredOperands(final int requiredOperands) + { + return requiredOperandCount(requiredOperands); + } + /** * Signifies that the operands at positions given by {@code literalOperands} must be literals. * * Must be used in conjunction with {@link #operandTypes(SqlTypeFamily...)}; this method is not compatible with * {@link #operandTypeChecker(SqlOperandTypeChecker)}. */ - public OperatorBuilder literalOperands(final int... literalOperands) + public OperatorBuilder literalOperands(final int... literalOperands) { this.literalOperands = literalOperands; return this; } - public OperatorBuilder operandTypeInference(SqlOperandTypeInference operandTypeInference) + public OperatorBuilder operandTypeInference(SqlOperandTypeInference operandTypeInference) { this.operandTypeInference = operandTypeInference; return this; } - public OperatorBuilder sqlKind(SqlKind kind) - { - this.kind = kind; - return this; - } - /** * Creates a {@link SqlFunction} from this builder. */ - public SqlFunction build() + @SuppressWarnings("unchecked") + public T build() + { + final IntSet nullableOperands = buildNullableOperands(); + return (T) new SqlFunction( + name, + kind, + Preconditions.checkNotNull(returnTypeInference, "returnTypeInference"), + buildOperandTypeInference(nullableOperands), + buildOperandTypeChecker(nullableOperands), + functionCategory + ); + } + + protected IntSet buildNullableOperands() { // Create "nullableOperands" set including all optional arguments. final IntSet nullableOperands = new IntArraySet(); - if (requiredOperands != null) { - IntStream.range(requiredOperands, operandTypes.size()).forEach(nullableOperands::add); + if (requiredOperandCount != null) { + IntStream.range(requiredOperandCount, operandTypes.size()).forEach(nullableOperands::add); } + return nullableOperands; + } - final SqlOperandTypeChecker theOperandTypeChecker; - + protected SqlOperandTypeChecker buildOperandTypeChecker(final IntSet nullableOperands) + { if (operandTypeChecker == null) { - theOperandTypeChecker = new DefaultOperandTypeChecker( + return new DefaultOperandTypeChecker( + operandNames, operandTypes, - requiredOperands == null ? operandTypes.size() : requiredOperands, + requiredOperandCount == null ? operandTypes.size() : requiredOperandCount, nullableOperands, literalOperands ); - } else if (operandTypes == null && requiredOperands == null && literalOperands == null) { - theOperandTypeChecker = operandTypeChecker; + } else if (operandNames.isEmpty() + && operandTypes == null + && requiredOperandCount == null + && literalOperands == null) { + return operandTypeChecker; } else { throw new ISE( - "Cannot have both 'operandTypeChecker' and 'operandTypes' / 'requiredOperands' / 'literalOperands'" + "Cannot have both 'operandTypeChecker' and " + + "'operandNames' / 'operandTypes' / 'requiredOperands' / 'literalOperands'" ); } + } + protected SqlOperandTypeInference buildOperandTypeInference(final IntSet nullableOperands) + { if (operandTypeInference == null) { SqlOperandTypeInference defaultInference = new DefaultOperandTypeInference(operandTypes, nullableOperands); - operandTypeInference = (callBinding, returnType, types) -> { + return (callBinding, returnType, types) -> { for (int i = 0; i < types.length; i++) { // calcite sql validate tries to do bad things to dynamic parameters if the type is inferred to be a string if (callBinding.operand(i).isA(ImmutableSet.of(SqlKind.DYNAMIC_PARAM))) { @@ -562,15 +617,49 @@ public class OperatorConversions } } }; + } else { + return operandTypeInference; } - return new SqlFunction( - name, - kind, - Preconditions.checkNotNull(returnTypeInference, "returnTypeInference"), - operandTypeInference, - theOperandTypeChecker, - functionCategory - ); + } + } + + public static class AggregatorBuilder extends OperatorBuilder + { + public AggregatorBuilder(String name) + { + super(name); + } + + /** + * Create a {@link SqlAggFunction} from this builder. + */ + @Override + public SqlAggFunction build() + { + final IntSet nullableOperands = buildNullableOperands(); + final SqlOperandTypeInference operandTypeInference = buildOperandTypeInference(nullableOperands); + final SqlOperandTypeChecker operandTypeChecker = buildOperandTypeChecker(nullableOperands); + + class DruidSqlAggFunction extends SqlAggFunction + { + public DruidSqlAggFunction() + { + super( + name, + null, + AggregatorBuilder.this.kind, + returnTypeInference, + operandTypeInference, + operandTypeChecker, + functionCategory, + false, + false, + Optionality.FORBIDDEN + ); + } + } + + return new DruidSqlAggFunction(); } } @@ -655,6 +744,11 @@ public class OperatorConversions @VisibleForTesting static class DefaultOperandTypeChecker implements SqlOperandTypeChecker { + /** + * Operand names for {@link #getAllowedSignatures(SqlOperator, String)}. May be empty, in which case the + * {@link #operandTypes} are used instead. + */ + private final List operandNames; private final List operandTypes; private final int requiredOperands; private final IntSet nullableOperands; @@ -662,6 +756,7 @@ public class OperatorConversions @VisibleForTesting DefaultOperandTypeChecker( + final List operandNames, final List operandTypes, final int requiredOperands, final IntSet nullableOperands, @@ -669,10 +764,15 @@ public class OperatorConversions ) { Preconditions.checkArgument(requiredOperands <= operandTypes.size() && requiredOperands >= 0); + this.operandNames = Preconditions.checkNotNull(operandNames, "operandNames"); this.operandTypes = Preconditions.checkNotNull(operandTypes, "operandTypes"); this.requiredOperands = requiredOperands; this.nullableOperands = Preconditions.checkNotNull(nullableOperands, "nullableOperands"); + if (!operandNames.isEmpty() && operandNames.size() != operandTypes.size()) { + throw new ISE("Operand name count[%s] and type count[%s] must match", operandNames.size(), operandTypes.size()); + } + if (literalOperands == null) { this.literalOperands = IntSets.EMPTY_SET; } else { @@ -688,8 +788,8 @@ public class OperatorConversions final SqlNode operand = callBinding.operands().get(i); if (literalOperands.contains(i)) { - // Verify that 'operand' is a literal. - if (!SqlUtil.isLiteral(operand)) { + // Verify that 'operand' is a literal. Allow CAST, since we can reduce these away later. + if (!SqlUtil.isLiteral(operand, true)) { return throwOrReturn( throwOnFailure, callBinding, @@ -739,7 +839,25 @@ public class OperatorConversions @Override public String getAllowedSignatures(SqlOperator op, String opName) { - return SqlUtil.getAliasedSignature(op, opName, operandTypes); + final List operands = !operandNames.isEmpty() ? operandNames : operandTypes; + final StringBuilder ret = new StringBuilder(); + ret.append("'"); + ret.append(opName); + ret.append("("); + for (int i = 0; i < operands.size(); i++) { + if (i > 0) { + ret.append(", "); + } + if (i >= requiredOperands) { + ret.append("["); + } + ret.append("<").append(operands.get(i)).append(">"); + } + for (int i = requiredOperands; i < operands.size(); i++) { + ret.append("]"); + } + ret.append(")'"); + return ret.toString(); } @Override @@ -772,7 +890,7 @@ public class OperatorConversions { return new DirectOperatorConversion( operatorBuilder(sqlOperator) - .requiredOperands(1) + .requiredOperandCount(1) .operandTypes(SqlTypeFamily.NUMERIC) .returnTypeNullable(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.NUMERIC) @@ -785,7 +903,7 @@ public class OperatorConversions { return new DirectOperatorConversion( operatorBuilder(sqlOperator) - .requiredOperands(2) + .requiredOperandCount(2) .operandTypes(SqlTypeFamily.NUMERIC, SqlTypeFamily.NUMERIC) .returnTypeNullable(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.NUMERIC) @@ -798,7 +916,7 @@ public class OperatorConversions { return new DirectOperatorConversion( operatorBuilder(StringUtils.toUpperCase(sqlOperator)) - .requiredOperands(1) + .requiredOperandCount(1) .operandTypes(SqlTypeFamily.NUMERIC) .returnTypeNullable(SqlTypeName.DOUBLE) .functionCategory(SqlFunctionCategory.NUMERIC) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BTrimOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BTrimOperatorConversion.java index d5a24773fe2..b8ae238dcf4 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BTrimOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BTrimOperatorConversion.java @@ -40,7 +40,7 @@ public class BTrimOperatorConversion implements SqlOperatorConversion .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) .returnTypeCascadeNullable(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) - .requiredOperands(1) + .requiredOperandCount(1) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ContainsOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ContainsOperatorConversion.java index 98703feb028..5e8071fb90c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ContainsOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ContainsOperatorConversion.java @@ -80,7 +80,7 @@ public class ContainsOperatorConversion extends DirectOperatorConversion return OperatorConversions .operatorBuilder(StringUtils.toUpperCase(functionName)) .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .requiredOperands(2) + .requiredOperandCount(2) .literalOperands(1) .returnTypeNonNull(SqlTypeName.BOOLEAN) .functionCategory(SqlFunctionCategory.STRING) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java index b913cd32caa..361de42862b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java @@ -67,7 +67,7 @@ public class DateTruncOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("DATE_TRUNC") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.TIMESTAMP) - .requiredOperands(2) + .requiredOperandCount(2) .returnTypeCascadeNullable(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java index 2dd89e7b7d6..8ccecafa3be 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java @@ -40,7 +40,7 @@ public class LPadOperatorConversion implements SqlOperatorConversion .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.CHARACTER) .returnTypeCascadeNullable(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) - .requiredOperands(2) + .requiredOperandCount(2) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LTrimOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LTrimOperatorConversion.java index 64d99e84a75..eef7584632e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LTrimOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LTrimOperatorConversion.java @@ -40,7 +40,7 @@ public class LTrimOperatorConversion implements SqlOperatorConversion .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) .returnTypeCascadeNullable(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) - .requiredOperands(1) + .requiredOperandCount(1) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java index 2bb00a22914..53b7b790256 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java @@ -40,7 +40,7 @@ public class ParseLongOperatorConversion implements SqlOperatorConversion .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER) .returnTypeCascadeNullable(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.STRING) - .requiredOperands(1) + .requiredOperandCount(1) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java index 68ee6537cd8..8e85d7a0682 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java @@ -40,7 +40,7 @@ public class RPadOperatorConversion implements SqlOperatorConversion .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.CHARACTER) .returnTypeCascadeNullable(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) - .requiredOperands(2) + .requiredOperandCount(2) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RTrimOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RTrimOperatorConversion.java index 70da7fbb4e6..bf5c9b0fe95 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RTrimOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RTrimOperatorConversion.java @@ -40,7 +40,7 @@ public class RTrimOperatorConversion implements SqlOperatorConversion .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) .returnTypeCascadeNullable(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) - .requiredOperands(1) + .requiredOperandCount(1) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java index 95c0e3421b0..804e5cad05d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java @@ -38,7 +38,7 @@ public class RegexpExtractOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("REGEXP_EXTRACT") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER) - .requiredOperands(2) + .requiredOperandCount(2) .literalOperands(1, 2) .returnTypeNullable(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpLikeOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpLikeOperatorConversion.java index b065b276cc5..cc677215cba 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpLikeOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpLikeOperatorConversion.java @@ -44,7 +44,7 @@ public class RegexpLikeOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("REGEXP_LIKE") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .requiredOperands(2) + .requiredOperandCount(2) .literalOperands(1) .returnTypeNonNull(SqlTypeName.BOOLEAN) .functionCategory(SqlFunctionCategory.STRING) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RoundOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RoundOperatorConversion.java index 76e5dc1da95..a14e3a0f0d9 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RoundOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RoundOperatorConversion.java @@ -35,7 +35,7 @@ public class RoundOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("ROUND") .operandTypes(SqlTypeFamily.NUMERIC, SqlTypeFamily.INTEGER) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeInference(ReturnTypes.ARG0) .functionCategory(SqlFunctionCategory.NUMERIC) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SubstringOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SubstringOperatorConversion.java index 4b9e35b7198..ae470922c1c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SubstringOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SubstringOperatorConversion.java @@ -43,7 +43,7 @@ public class SubstringOperatorConversion implements SqlOperatorConversion .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER) .functionCategory(SqlFunctionCategory.STRING) .returnTypeInference(ReturnTypes.ARG0) - .requiredOperands(2) + .requiredOperandCount(2) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TextcatOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TextcatOperatorConversion.java index 5da41c42c80..40a13479b8c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TextcatOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TextcatOperatorConversion.java @@ -31,7 +31,7 @@ public class TextcatOperatorConversion extends DirectOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("textcat") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .requiredOperands(2) + .requiredOperandCount(2) .returnTypeCascadeNullable(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeCeilOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeCeilOperatorConversion.java index 75680050528..ffa022b0ee8 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeCeilOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeCeilOperatorConversion.java @@ -41,7 +41,7 @@ public class TimeCeilOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("TIME_CEIL") .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER) - .requiredOperands(2) + .requiredOperandCount(2) .returnTypeCascadeNullable(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeExtractOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeExtractOperatorConversion.java index 69397bc4d3a..6a36bf4dad0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeExtractOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeExtractOperatorConversion.java @@ -43,7 +43,7 @@ public class TimeExtractOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("TIME_EXTRACT") .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .requiredOperands(2) + .requiredOperandCount(2) .returnTypeCascadeNullable(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java index 9f74e87c797..b0b5b69acc7 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java @@ -56,7 +56,7 @@ public class TimeFloorOperatorConversion implements SqlOperatorConversion public static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(SQL_FUNCTION_NAME) .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER) - .requiredOperands(2) + .requiredOperandCount(2) .returnTypeCascadeNullable(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFormatOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFormatOperatorConversion.java index 8c07ffdf4ea..60127fc2099 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFormatOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFormatOperatorConversion.java @@ -46,7 +46,7 @@ public class TimeFormatOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("TIME_FORMAT") .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeCascadeNullable(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeParseOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeParseOperatorConversion.java index 99bc94cc8f1..7e515a35168 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeParseOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeParseOperatorConversion.java @@ -43,7 +43,7 @@ public class TimeParseOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("TIME_PARSE") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNullable(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeShiftOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeShiftOperatorConversion.java index d2973fb6d18..edda4d07406 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeShiftOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeShiftOperatorConversion.java @@ -43,7 +43,7 @@ public class TimeShiftOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("TIME_SHIFT") .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.CHARACTER) - .requiredOperands(3) + .requiredOperandCount(3) .returnTypeCascadeNullable(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TruncateOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TruncateOperatorConversion.java index 8459bed7d72..d891823dd15 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TruncateOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TruncateOperatorConversion.java @@ -37,7 +37,7 @@ public class TruncateOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("TRUNCATE") .operandTypes(SqlTypeFamily.NUMERIC, SqlTypeFamily.INTEGER) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeInference(ReturnTypes.ARG0) .functionCategory(SqlFunctionCategory.NUMERIC) .build(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/OperatorConversionsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/OperatorConversionsTest.java index 5f70dc5902a..d3f97eed22b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/OperatorConversionsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/OperatorConversionsTest.java @@ -51,6 +51,7 @@ import org.mockito.stubbing.Answer; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; @RunWith(Enclosed.class) @@ -65,6 +66,7 @@ public class OperatorConversionsTest public void testGetOperandCountRange() { SqlOperandTypeChecker typeChecker = new DefaultOperandTypeChecker( + Collections.emptyList(), ImmutableList.of(SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER), 2, IntSets.EMPTY_SET, @@ -79,6 +81,7 @@ public class OperatorConversionsTest public void testIsOptional() { SqlOperandTypeChecker typeChecker = new DefaultOperandTypeChecker( + Collections.emptyList(), ImmutableList.of(SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER), 2, IntSets.EMPTY_SET, @@ -95,7 +98,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testAllowFullOperands") .operandTypes(SqlTypeFamily.INTEGER, SqlTypeFamily.DATE) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -119,7 +122,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testRequiredOperandsOnly") .operandTypes(SqlTypeFamily.INTEGER, SqlTypeFamily.DATE) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -140,7 +143,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testLiteralOperandCheckLiteral") .operandTypes(SqlTypeFamily.INTEGER) - .requiredOperands(1) + .requiredOperandCount(1) .literalOperands(0) .returnTypeNonNull(SqlTypeName.CHAR) .build(); @@ -162,7 +165,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testLiteralOperandCheckLiteralThrow") .operandTypes(SqlTypeFamily.INTEGER) - .requiredOperands(1) + .requiredOperandCount(1) .literalOperands(0) .returnTypeNonNull(SqlTypeName.CHAR) .build(); @@ -184,7 +187,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testAnyTypeOperand") .operandTypes(SqlTypeFamily.ANY) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -205,7 +208,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testCastableFromDatetimeFamilyToTimestamp") .operandTypes(SqlTypeFamily.DATETIME) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -235,7 +238,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testNullForNullableOperand") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTERVAL_DAY_TIME) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -259,7 +262,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testNullLiteralForNullableOperand") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTERVAL_DAY_TIME) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -283,7 +286,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testNullForNullableNonnull") .operandTypes(SqlTypeFamily.CHARACTER) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -304,7 +307,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testNullForNullableCascade") .operandTypes(SqlTypeFamily.CHARACTER) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeCascadeNullable(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -325,7 +328,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testNullForNullableNonnull") .operandTypes(SqlTypeFamily.CHARACTER) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNullable(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -346,7 +349,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testNullForNonNullableOperand") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTERVAL_DAY_TIME) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -372,7 +375,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testNullLiteralForNonNullableOperand") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTERVAL_DAY_TIME) - .requiredOperands(1) + .requiredOperandCount(1) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -398,7 +401,7 @@ public class OperatorConversionsTest SqlFunction function = OperatorConversions .operatorBuilder("testNonCastableType") .operandTypes(SqlTypeFamily.CURSOR, SqlTypeFamily.INTERVAL_DAY_TIME) - .requiredOperands(2) + .requiredOperandCount(2) .returnTypeNonNull(SqlTypeName.CHAR) .build(); SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); @@ -418,6 +421,41 @@ public class OperatorConversionsTest ); } + @Test + public void testSignatureWithNames() + { + SqlFunction function = OperatorConversions + .operatorBuilder("testSignatureWithNames") + .operandNames("x", "y", "z") + .operandTypes(SqlTypeFamily.INTEGER, SqlTypeFamily.DATE, SqlTypeFamily.ANY) + .requiredOperandCount(1) + .returnTypeNonNull(SqlTypeName.CHAR) + .build(); + SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); + + Assert.assertEquals( + "'testSignatureWithNames(, [, []])'", + typeChecker.getAllowedSignatures(function, function.getName()) + ); + } + + @Test + public void testSignatureWithoutNames() + { + SqlFunction function = OperatorConversions + .operatorBuilder("testSignatureWithoutNames") + .operandTypes(SqlTypeFamily.INTEGER, SqlTypeFamily.DATE, SqlTypeFamily.ANY) + .requiredOperandCount(1) + .returnTypeNonNull(SqlTypeName.CHAR) + .build(); + SqlOperandTypeChecker typeChecker = function.getOperandTypeChecker(); + + Assert.assertEquals( + "'testSignatureWithoutNames(, [, []])'", + typeChecker.getAllowedSignatures(function, function.getName()) + ); + } + private static SqlCallBinding mockCallBinding( SqlFunction function, List actualOperands diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidOperatorTableTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidOperatorTableTest.java index 1e148517631..02e5c522f27 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidOperatorTableTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidOperatorTableTest.java @@ -68,7 +68,7 @@ public class DruidOperatorTableTest final SqlOperator operator1 = OperatorConversions .operatorBuilder("FOO") .operandTypes(SqlTypeFamily.ANY) - .requiredOperands(0) + .requiredOperandCount(0) .returnTypeInference( opBinding -> RowSignatures.makeComplexType( opBinding.getTypeFactory(), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java index bb5edf2d94c..205f3379d71 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java @@ -69,7 +69,7 @@ public class DruidRexExecutorTest extends InitializedNullHandlingTest private static final SqlOperator OPERATOR = OperatorConversions .operatorBuilder(StringUtils.toUpperCase("hyper_unique")) .operandTypes(SqlTypeFamily.ANY) - .requiredOperands(0) + .requiredOperandCount(0) .returnTypeInference( opBinding -> RowSignatures.makeComplexType( opBinding.getTypeFactory(), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/InformationSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/InformationSchemaTest.java index 306599b2f32..43551b3e040 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/InformationSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/InformationSchemaTest.java @@ -136,7 +136,7 @@ public class InformationSchemaTest extends BaseCalciteQueryTest Assert.assertNotNull(rows); Assert.assertEquals("There should be exactly 2 rows; any non-function syntax operator should get filtered out", 2, rows.size()); - Object[] expectedRow1 = {"druid", "INFORMATION_SCHEMA", "FOO", "FUNCTION", "NO", "'FOO()'"}; + Object[] expectedRow1 = {"druid", "INFORMATION_SCHEMA", "FOO", "FUNCTION", "NO", "'FOO([])'"}; Assert.assertTrue(rows.stream().anyMatch(row -> Arrays.equals(row, expectedRow1))); Object[] expectedRow2 = {"druid", "INFORMATION_SCHEMA", "BAR", "FUNCTION", "NO", "'BAR(, )'"}; @@ -166,7 +166,7 @@ public class InformationSchemaTest extends BaseCalciteQueryTest final SqlOperator operator1 = OperatorConversions .operatorBuilder("FOO") .operandTypes(SqlTypeFamily.ANY) - .requiredOperands(0) + .requiredOperandCount(0) .returnTypeInference( opBinding -> RowSignatures.makeComplexType( opBinding.getTypeFactory(), @@ -182,7 +182,7 @@ public class InformationSchemaTest extends BaseCalciteQueryTest .operatorBuilder("BAR") .operandTypes(SqlTypeFamily.NUMERIC) .operandTypes(SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER) - .requiredOperands(2) + .requiredOperandCount(2) .returnTypeInference( opBinding -> RowSignatures.makeComplexType( opBinding.getTypeFactory(), From 970288067ac2979dff0ddeed97003cd631ad9ee8 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 23 Jun 2023 16:27:11 -0700 Subject: [PATCH 23/74] Fix flaky HttpEmitterConfigTest and ParametrizedUriEmitterConfigTest. (#14481) Recently, we have seen flakiness in these two tests, apparently due to computations based on Runtime.getRuntime().maxMemory() differing during static initialization and in the actual tests. I can't think of a reason why this would be happening, but anyway, this patch switches the tests to use the statics instead of recomputing Runtime.getRuntime().maxMemory(). --- .../util/emitter/core/HttpEmitterConfigTest.java | 15 ++++----------- .../core/ParametrizedUriEmitterConfigTest.java | 8 ++------ 2 files changed, 6 insertions(+), 17 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfigTest.java index 1191b83b73e..ebc213303db 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/HttpEmitterConfigTest.java @@ -21,7 +21,6 @@ package org.apache.druid.java.util.emitter.core; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.utils.JvmUtils; import org.junit.Assert; import org.junit.Test; @@ -46,11 +45,8 @@ public class HttpEmitterConfigTest Assert.assertEquals("http://example.com/", config.getRecipientBaseUrl()); Assert.assertNull(config.getBasicAuthentication()); Assert.assertEquals(BatchingStrategy.ARRAY, config.getBatchingStrategy()); - Pair batchConfigPair = BaseHttpEmittingConfig.getDefaultBatchSizeAndLimit( - JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() - ); - Assert.assertEquals(batchConfigPair.lhs.intValue(), config.getMaxBatchSize()); - Assert.assertEquals(batchConfigPair.rhs.intValue(), config.getBatchQueueSizeLimit()); + Assert.assertEquals(BaseHttpEmittingConfig.DEFAULT_MAX_BATCH_SIZE, config.getMaxBatchSize()); + Assert.assertEquals(BaseHttpEmittingConfig.DEFAULT_BATCH_QUEUE_SIZE_LIMIT, config.getBatchQueueSizeLimit()); Assert.assertEquals(Long.MAX_VALUE, config.getFlushTimeOut()); Assert.assertEquals(2.0f, config.getHttpTimeoutAllowanceFactor(), 0.0f); Assert.assertEquals(0, config.getMinHttpTimeoutMillis()); @@ -70,11 +66,8 @@ public class HttpEmitterConfigTest Assert.assertEquals("http://example.com/", config.getRecipientBaseUrl()); Assert.assertNull(config.getBasicAuthentication()); Assert.assertEquals(BatchingStrategy.ARRAY, config.getBatchingStrategy()); - Pair batchConfigPair = BaseHttpEmittingConfig.getDefaultBatchSizeAndLimit( - JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() - ); - Assert.assertEquals(batchConfigPair.lhs.intValue(), config.getMaxBatchSize()); - Assert.assertEquals(batchConfigPair.rhs.intValue(), config.getBatchQueueSizeLimit()); + Assert.assertEquals(BaseHttpEmittingConfig.DEFAULT_MAX_BATCH_SIZE, config.getMaxBatchSize()); + Assert.assertEquals(BaseHttpEmittingConfig.DEFAULT_BATCH_QUEUE_SIZE_LIMIT, config.getBatchQueueSizeLimit()); Assert.assertEquals(Long.MAX_VALUE, config.getFlushTimeOut()); Assert.assertEquals(2.0f, config.getHttpTimeoutAllowanceFactor(), 0.0f); Assert.assertEquals(0, config.getMinHttpTimeoutMillis()); diff --git a/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfigTest.java b/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfigTest.java index 53740a79141..e99b1d4f9d6 100644 --- a/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfigTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitterConfigTest.java @@ -22,7 +22,6 @@ package org.apache.druid.java.util.emitter.core; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Guice; import com.google.inject.Injector; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.utils.JvmUtils; import org.apache.druid.utils.RuntimeInfo; import org.junit.AfterClass; @@ -67,11 +66,8 @@ public class ParametrizedUriEmitterConfigTest final Injector injector = makeInjector(new Properties()); final HttpEmitterConfig config = injector.getInstance(HttpEmitterConfig.class); - Pair batchConfigPair = BaseHttpEmittingConfig.getDefaultBatchSizeAndLimit( - JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() - ); - Assert.assertEquals(batchConfigPair.lhs.intValue(), config.getMaxBatchSize()); - Assert.assertEquals(batchConfigPair.rhs.intValue(), config.getBatchQueueSizeLimit()); + Assert.assertEquals(BaseHttpEmittingConfig.DEFAULT_MAX_BATCH_SIZE, config.getMaxBatchSize()); + Assert.assertEquals(BaseHttpEmittingConfig.DEFAULT_BATCH_QUEUE_SIZE_LIMIT, config.getBatchQueueSizeLimit()); Assert.assertEquals(Long.MAX_VALUE, config.getFlushTimeOut()); } From 72cf91fbc0ed9cc2abce91df878ab431678b12f3 Mon Sep 17 00:00:00 2001 From: Tejaswini Bandlamudi <96047043+tejaswini-imply@users.noreply.github.com> Date: Sat, 24 Jun 2023 14:51:30 +0530 Subject: [PATCH 24/74] Upgrade Avro to latest version (#14440) Upgraded Avro to 1.11.1 --- .../apache/druid/data/input/avro/AvroFlattenerMaker.java | 6 +++++- .../druid/data/input/avro/AvroFlattenerMakerTest.java | 4 ++++ licenses.yaml | 4 ++-- owasp-dependency-check-suppressions.xml | 8 -------- pom.xml | 2 +- 5 files changed, 12 insertions(+), 12 deletions(-) diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java index d557b65c847..007608159f4 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java @@ -141,7 +141,11 @@ public class AvroFlattenerMaker implements ObjectFlatteners.FlattenerMakerCVE-2020-13949 - - - - CVE-2020-13936 - - 2.2.4 2.13.9 1.17.0 - 1.9.2 + 1.11.1 -This topic documents all of the API endpoints for each Druid service type. - -## Common - -All processes support the following endpoints. - -### Process information - -`GET /status` - -Returns the Druid version, loaded extensions, memory used, total memory, and other useful information about the process. - -`GET /status/health` - -Always returns a boolean `true` value with a 200 OK response, useful for automated health checks. - -`GET /status/properties` - -Returns the current configuration properties of the process. - -`GET /status/selfDiscovered/status` - -Returns a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has received a confirmation -from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the -cluster. It is recommended to not consider a Druid node "healthy" or "ready" in automated deployment/container -management systems until it returns `{"selfDiscovered": true}` from this endpoint. This is because a node may be -isolated from the rest of the cluster due to network issues and it doesn't make sense to consider nodes "healthy" in -this case. Also, when nodes such as Brokers use ZooKeeper segment discovery for building their view of the Druid cluster -(as opposed to HTTP segment discovery), they may be unusable until the ZooKeeper client is fully initialized and starts -to receive data from the ZooKeeper cluster. `{"selfDiscovered": true}` is a proxy event indicating that the ZooKeeper -client on the node has started to receive data from the ZooKeeper cluster and it's expected that all segments and other -nodes will be discovered by this node timely from this point. - -`GET /status/selfDiscovered` - -Similar to `/status/selfDiscovered/status`, but returns 200 OK response with empty body if the node has discovered itself -and 503 SERVICE UNAVAILABLE if the node hasn't discovered itself yet. This endpoint might be useful because some -monitoring checks such as AWS load balancer health checks are not able to look at the response body. - -## Master server - -This section documents the API endpoints for the processes that reside on Master servers (Coordinators and Overlords) -in the suggested [three-server configuration](../design/processes.md#server-types). - -### Coordinator - -#### Leadership - -`GET /druid/coordinator/v1/leader` - -Returns the current leader Coordinator of the cluster. - -`GET /druid/coordinator/v1/isLeader` - -Returns a JSON object with `leader` parameter, either true or false, indicating if this server is the current leader -Coordinator of the cluster. In addition, returns HTTP 200 if the server is the current leader and HTTP 404 if not. -This is suitable for use as a load balancer status check if you only want the active leader to be considered in-service -at the load balancer. - - - - -#### Segment loading - -`GET /druid/coordinator/v1/loadstatus` - -Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster. - -`GET /druid/coordinator/v1/loadstatus?simple` - -Returns the number of segments left to load until segments that should be loaded in the cluster are available for queries. This does not include segment replication counts. - -`GET /druid/coordinator/v1/loadstatus?full` - -Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts. - -`GET /druid/coordinator/v1/loadstatus?full&computeUsingClusterView` - -Returns the number of segments not yet loaded for each tier until all segments loading in the cluster are available. -The result includes segment replication counts. It also factors in the number of available nodes that are of a service type that can load the segment when computing the number of segments remaining to load. -A segment is considered fully loaded when: -- Druid has replicated it the number of times configured in the corresponding load rule. -- Or the number of replicas for the segment in each tier where it is configured to be replicated equals the available nodes of a service type that are currently allowed to load the segment in the tier. - -`GET /druid/coordinator/v1/loadqueue` - -Returns the ids of segments to load and drop for each Historical process. - -`GET /druid/coordinator/v1/loadqueue?simple` - -Returns the number of segments to load and drop, as well as the total segment load and drop size in bytes for each Historical process. - -`GET /druid/coordinator/v1/loadqueue?full` - -Returns the serialized JSON of segments to load and drop for each Historical process. - -#### Segment loading by datasource - -Note that all _interval_ query parameters are ISO 8601 strings—for example, 2016-06-27/2016-06-28. -Also note that these APIs only guarantees that the segments are available at the time of the call. -Segments can still become missing because of historical process failures or any other reasons afterward. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?forceMetadataRefresh={boolean}&interval={myInterval}` - -Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster for the given -datasource over the given interval (or last 2 weeks if interval is not given). `forceMetadataRefresh` is required to be set. -* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store -(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms -of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status) -* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. -If no used segments are found for the given inputs, this API returns `204 No Content` - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?simple&forceMetadataRefresh={boolean}&interval={myInterval}` - -Returns the number of segments left to load until segments that should be loaded in the cluster are available for the given datasource -over the given interval (or last 2 weeks if interval is not given). This does not include segment replication counts. `forceMetadataRefresh` is required to be set. -* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store -(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms -of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status) -* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. -If no used segments are found for the given inputs, this API returns `204 No Content` - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?full&forceMetadataRefresh={boolean}&interval={myInterval}` - -Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available for the given datasource over the given interval (or last 2 weeks if interval is not given). This includes segment replication counts. `forceMetadataRefresh` is required to be set. -* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store -(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms -of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status) -* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. - -You can pass the optional query parameter `computeUsingClusterView` to factor in the available cluster services when calculating -the segments left to load. See [Coordinator Segment Loading](#coordinator-segment-loading) for details. -If no used segments are found for the given inputs, this API returns `204 No Content` - -#### Metadata store information - -> Note: Much of this information is available in a simpler, easier-to-use form through the Druid SQL -> [`sys.segments`](../querying/sql-metadata-tables.md#segments-table) table. - -`GET /druid/coordinator/v1/metadata/segments` - -Returns a list of all segments for each datasource enabled in the cluster. - -`GET /druid/coordinator/v1/metadata/segments?datasources={dataSourceName1}&datasources={dataSourceName2}` - -Returns a list of all segments for one or more specific datasources enabled in the cluster. - -`GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus` - -Returns a list of all segments for each datasource with the full segment metadata and extra fields `overshadowed` and `replicationFactor`. - -`GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&datasources={dataSourceName1}&datasources={dataSourceName2}` - -Returns a list of all segments for one or more specific datasources with the full segment metadata and extra fields `overshadowed` and `replicationFactor`. - -`GET /druid/coordinator/v1/metadata/datasources` - -Returns a list of the names of datasources with at least one used segment in the cluster, retrieved from the metadata database. Users should call this API to get the eventual state that the system will be in. - -`GET /druid/coordinator/v1/metadata/datasources?includeUnused` - -Returns a list of the names of datasources, regardless of whether there are used segments belonging to those datasources in the cluster or not. - -`GET /druid/coordinator/v1/metadata/datasources?includeDisabled` - -Returns a list of the names of datasources, regardless of whether the datasource is disabled or not. - -`GET /druid/coordinator/v1/metadata/datasources?full` - -Returns a list of all datasources with at least one used segment in the cluster. Returns all metadata about those datasources as stored in the metadata store. - -`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}` - -Returns full metadata for a datasource as stored in the metadata store. - -`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments` - -Returns a list of all segments for a datasource as stored in the metadata store. - -`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full` - -Returns a list of all segments for a datasource with the full segment metadata as stored in the metadata store. - -`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments/{segmentId}` - -Returns full segment metadata for a specific segment as stored in the metadata store, if the segment is used. If the -segment is unused, or is unknown, a 404 response is returned. - -##### POST - -`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments` - -Returns a list of all segments, overlapping with any of given intervals, for a datasource as stored in the metadata store. Request body is array of string IS0 8601 intervals like `[interval1, interval2,...]`—for example, `["2012-01-01T00:00:00.000/2012-01-03T00:00:00.000", "2012-01-05T00:00:00.000/2012-01-07T00:00:00.000"]`. - -`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full` - -Returns a list of all segments, overlapping with any of given intervals, for a datasource with the full segment metadata as stored in the metadata store. Request body is array of string ISO 8601 intervals like `[interval1, interval2,...]`—for example, `["2012-01-01T00:00:00.000/2012-01-03T00:00:00.000", "2012-01-05T00:00:00.000/2012-01-07T00:00:00.000"]`. - - - -#### Datasources - -Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`—for example, `2016-06-27_2016-06-28`. - -`GET /druid/coordinator/v1/datasources` - -Returns a list of datasource names found in the cluster as seen by the coordinator. This view is updated every [`druid.coordinator.period`](../configuration/index.md#coordinator-operation). - -`GET /druid/coordinator/v1/datasources?simple` - -Returns a list of JSON objects containing the name and properties of datasources found in the cluster. Properties include segment count, total segment byte size, replicated total segment byte size, minTime, and maxTime. - -`GET /druid/coordinator/v1/datasources?full` - -Returns a list of datasource names found in the cluster with all metadata about those datasources. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}` - -Returns a JSON object containing the name and properties of a datasource. Properties include segment count, total segment byte size, replicated total segment byte size, minTime, and maxTime. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}?full` - -Returns full metadata for a datasource. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals` - -Returns a set of segment intervals. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals?simple` - -Returns a map of an interval to a JSON object containing the total byte size of segments and number of segments for that interval. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals?full` - -Returns a map of an interval to a map of segment metadata to a set of server names that contain the segment for that interval. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}` - -Returns a set of segment ids for an interval. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?simple` - -Returns a map of segment intervals contained within the specified interval to a JSON object containing the total byte size of segments and number of segments for an interval. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?full` - -Returns a map of segment intervals contained within the specified interval to a map of segment metadata to a set of server names that contain the segment for an interval. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}/serverview` - -Returns a map of segment intervals contained within the specified interval to information about the servers that contain the segment for an interval. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments` - -Returns a list of all segments for a datasource in the cluster. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments?full` - -Returns a list of all segments for a datasource in the cluster with the full segment metadata. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` - -Returns full segment metadata for a specific segment in the cluster. - -`GET /druid/coordinator/v1/datasources/{dataSourceName}/tiers` - -Return the tiers that a datasource exists in. - -#### Note for Coordinator's POST and DELETE APIs - -While segments may be enabled by issuing POST requests for the datasources, the Coordinator may again disable segments if they match any configured [drop rules](../operations/rule-configuration.md#drop-rules). Even if segments are enabled by these APIs, you must configure a [load rule](../operations/rule-configuration.md#load-rules) to load them onto Historical processes. If an indexing or kill task runs at the same time these APIs are invoked, the behavior is undefined. Some segments might be killed and others might be enabled. It's also possible that all segments might be disabled, but the indexing task can still read data from those segments and succeed. - -> Avoid using indexing or kill tasks and these APIs at the same time for the same datasource and time chunk. - -`POST /druid/coordinator/v1/datasources/{dataSourceName}` - -Marks as used all segments belonging to a datasource. Returns a JSON object of the form -`{"numChangedSegments": }` with the number of segments in the database whose state has been changed (that is, -the segments were marked as used) as the result of this API call. - -`POST /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` - -Marks as used a segment of a datasource. Returns a JSON object of the form `{"segmentStateChanged": }` with -the boolean indicating if the state of the segment has been changed (that is, the segment was marked as used) as the -result of this API call. - -`POST /druid/coordinator/v1/datasources/{dataSourceName}/markUsed` - -`POST /druid/coordinator/v1/datasources/{dataSourceName}/markUnused` - -Marks segments (un)used for a datasource by interval or set of segment Ids. When marking used only segments that are not overshadowed will be updated. - -The request payload contains the interval or set of segment IDs to be marked unused. -Either interval or segment IDs should be provided, if both or none are provided in the payload, the API would throw an error (400 BAD REQUEST). - -Interval specifies the start and end times as IS0 8601 strings. `interval=(start/end)` where start and end both are inclusive and only the segments completely contained within the specified interval will be disabled, partially overlapping segments will not be affected. - -JSON Request Payload: - - |Key|Description|Example| -|----------|-------------|---------| -|`interval`|The interval for which to mark segments unused|`"2015-09-12T03:00:00.000Z/2015-09-12T05:00:00.000Z"`| -|`segmentIds`|Set of segment IDs to be marked unused|`["segmentId1", "segmentId2"]`| - -`DELETE /druid/coordinator/v1/datasources/{dataSourceName}` - -Marks as unused all segments belonging to a datasource. Returns a JSON object of the form -`{"numChangedSegments": }` with the number of segments in the database whose state has been changed (that is, -the segments were marked as unused) as the result of this API call. - -`DELETE /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}` -`@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myInterval}` - -Runs a [Kill task](../ingestion/tasks.md) for a given interval and datasource. - -`DELETE /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` - -Marks as unused a segment of a datasource. Returns a JSON object of the form `{"segmentStateChanged": }` with -the boolean indicating if the state of the segment has been changed (that is, the segment was marked as unused) as the -result of this API call. - -#### Retention rules - -Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` as in `2016-06-27_2016-06-28`. - -`GET /druid/coordinator/v1/rules` - -Returns all rules as JSON objects for all datasources in the cluster including the default datasource. - -`GET /druid/coordinator/v1/rules/{dataSourceName}` - -Returns all rules for a specified datasource. - -`GET /druid/coordinator/v1/rules/{dataSourceName}?full` - -Returns all rules for a specified datasource and includes default datasource. - -`GET /druid/coordinator/v1/rules/history?interval=` - -Returns audit history of rules for all datasources. Default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Coordinator `runtime.properties`. - -`GET /druid/coordinator/v1/rules/history?count=` - -Returns last `n` entries of audit history of rules for all datasources. - -`GET /druid/coordinator/v1/rules/{dataSourceName}/history?interval=` - -Returns audit history of rules for a specified datasource. Default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Coordinator `runtime.properties`. - -`GET /druid/coordinator/v1/rules/{dataSourceName}/history?count=` - -Returns last `n` entries of audit history of rules for a specified datasource. - -`POST /druid/coordinator/v1/rules/{dataSourceName}` - -POST with a list of rules in JSON form to update rules. - -Optional Header Parameters for auditing the config change can also be specified. - -|Header Param Name| Description | Default | -|----------|-------------|---------| -|`X-Druid-Author`| Author making the config change|`""`| -|`X-Druid-Comment`| Comment describing the change being done|`""`| - -#### Intervals - -Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` as in `2016-06-27_2016-06-28`. - -`GET /druid/coordinator/v1/intervals` - -Returns all intervals for all datasources with total size and count. - -`GET /druid/coordinator/v1/intervals/{interval}` - -Returns aggregated total size and count for all intervals that intersect given ISO interval. - -`GET /druid/coordinator/v1/intervals/{interval}?simple` - -Returns total size and count for each interval within given ISO interval. - -`GET /druid/coordinator/v1/intervals/{interval}?full` - -Returns total size and count for each datasource for each interval within given ISO interval. - -#### Dynamic configuration - -See [Coordinator Dynamic Configuration](../configuration/index.md#dynamic-configuration) for details. - -Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` -as in `2016-06-27_2016-06-28`. - -`GET /druid/coordinator/v1/config` - -Retrieves current coordinator dynamic configuration. - -`GET /druid/coordinator/v1/config/history?interval={interval}&count={count}` - -Retrieves history of changes to overlord dynamic configuration. Accepts `interval` and `count` query string parameters -to filter by interval and limit the number of results respectively. - -`POST /druid/coordinator/v1/config` - -Update overlord dynamic worker configuration. - -#### Automatic compaction status - -`GET /druid/coordinator/v1/compaction/progress?dataSource={dataSource}` - -Returns the total size of segments awaiting compaction for the given dataSource. The specified dataSource must have [automatic compaction](../data-management/automatic-compaction.md) enabled. - -`GET /druid/coordinator/v1/compaction/status` - -Returns the status and statistics from the auto-compaction run of all dataSources which have auto-compaction enabled in the latest run. The response payload includes a list of `latestStatus` objects. Each `latestStatus` represents the status for a dataSource (which has/had auto-compaction enabled). - -The `latestStatus` object has the following keys: -* `dataSource`: name of the datasource for this status information -* `scheduleStatus`: auto-compaction scheduling status. Possible values are `NOT_ENABLED` and `RUNNING`. Returns `RUNNING ` if the dataSource has an active auto-compaction config submitted. Otherwise, returns `NOT_ENABLED`. -* `bytesAwaitingCompaction`: total bytes of this datasource waiting to be compacted by the auto-compaction (only consider intervals/segments that are eligible for auto-compaction) -* `bytesCompacted`: total bytes of this datasource that are already compacted with the spec set in the auto-compaction config -* `bytesSkipped`: total bytes of this datasource that are skipped (not eligible for auto-compaction) by the auto-compaction -* `segmentCountAwaitingCompaction`: total number of segments of this datasource waiting to be compacted by the auto-compaction (only consider intervals/segments that are eligible for auto-compaction) -* `segmentCountCompacted`: total number of segments of this datasource that are already compacted with the spec set in the auto-compaction config -* `segmentCountSkipped`: total number of segments of this datasource that are skipped (not eligible for auto-compaction) by the auto-compaction -* `intervalCountAwaitingCompaction`: total number of intervals of this datasource waiting to be compacted by the auto-compaction (only consider intervals/segments that are eligible for auto-compaction) -* `intervalCountCompacted`: total number of intervals of this datasource that are already compacted with the spec set in the auto-compaction config -* `intervalCountSkipped`: total number of intervals of this datasource that are skipped (not eligible for auto-compaction) by the auto-compaction - -`GET /druid/coordinator/v1/compaction/status?dataSource={dataSource}` - -Similar to the API `/druid/coordinator/v1/compaction/status` above but filters response to only return information for the dataSource given. -The dataSource must have auto-compaction enabled. - -#### Automatic compaction configuration - -`GET /druid/coordinator/v1/config/compaction` - -Returns all automatic compaction configs. - -`GET /druid/coordinator/v1/config/compaction/{dataSource}` - -Returns an automatic compaction config of a dataSource. - -`GET /druid/coordinator/v1/config/compaction/{dataSource}/history?interval={interval}&count={count}` - -Returns the history of the automatic compaction config for a dataSource. Optionally accepts `interval` and `count` -query string parameters to filter by interval and limit the number of results respectively. If the dataSource does not -exist or there is no compaction history for the dataSource, an empty list is returned. - -The response contains a list of objects with the following keys: -* `globalConfig`: A json object containing automatic compaction config that applies to the entire cluster. -* `compactionConfig`: A json object containing the automatic compaction config for the datasource. -* `auditInfo`: A json object that contains information about the change made - like `author`, `comment` and `ip`. -* `auditTime`: The date and time when the change was made. - -`POST /druid/coordinator/v1/config/compaction/taskslots?ratio={someRatio}&max={someMaxSlots}` - -Update the capacity for compaction tasks. `ratio` and `max` are used to limit the max number of compaction tasks. -They mean the ratio of the total task slots to the compaction task slots and the maximum number of task slots for compaction tasks, respectively. The actual max number of compaction tasks is `min(max, ratio * total task slots)`. -Note that `ratio` and `max` are optional and can be omitted. If they are omitted, default values (0.1 and unbounded) -will be set for them. - -`POST /druid/coordinator/v1/config/compaction` - -Creates or updates the [automatic compaction](../data-management/automatic-compaction.md) config for a dataSource. See [Automatic compaction dynamic configuration](../configuration/index.md#automatic-compaction-dynamic-configuration) for configuration details. - -`DELETE /druid/coordinator/v1/config/compaction/{dataSource}` - -Removes the automatic compaction config for a dataSource. - -#### Server information - -`GET /druid/coordinator/v1/servers` - -Returns a list of servers URLs using the format `{hostname}:{port}`. Note that -processes that run with different types will appear multiple times with different -ports. - -`GET /druid/coordinator/v1/servers?simple` - -Returns a list of server data objects in which each object has the following keys: -* `host`: host URL include (`{hostname}:{port}`) -* `type`: process type (`indexer-executor`, `historical`) -* `currSize`: storage size currently used -* `maxSize`: maximum storage size -* `priority` -* `tier` - -### Overlord - -#### Leadership - -`GET /druid/indexer/v1/leader` - -Returns the current leader Overlord of the cluster. If you have multiple Overlords, just one is leading at any given time. The others are on standby. - -`GET /druid/indexer/v1/isLeader` - -This returns a JSON object with field `leader`, either true or false. In addition, this call returns HTTP 200 if the -server is the current leader and HTTP 404 if not. This is suitable for use as a load balancer status check if you -only want the active leader to be considered in-service at the load balancer. - -#### Tasks - -Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` -as in `2016-06-27_2016-06-28`. - -`GET /druid/indexer/v1/tasks` - -Retrieve list of tasks. Accepts query string parameters `state`, `datasource`, `createdTimeInterval`, `max`, and `type`. - -|Query Parameter |Description | -|---|---| -|`state`|filter list of tasks by task state, valid options are `running`, `complete`, `waiting`, and `pending`.| -| `datasource`| return tasks filtered by Druid datasource.| -| `createdTimeInterval`| return tasks created within the specified interval. | -| `max`| maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| -| `type`| filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| - - -`GET /druid/indexer/v1/completeTasks` - -Retrieve list of complete tasks. Equivalent to `/druid/indexer/v1/tasks?state=complete`. - -`GET /druid/indexer/v1/runningTasks` - -Retrieve list of running tasks. Equivalent to `/druid/indexer/v1/tasks?state=running`. - -`GET /druid/indexer/v1/waitingTasks` - -Retrieve list of waiting tasks. Equivalent to `/druid/indexer/v1/tasks?state=waiting`. - -`GET /druid/indexer/v1/pendingTasks` - -Retrieve list of pending tasks. Equivalent to `/druid/indexer/v1/tasks?state=pending`. - -`GET /druid/indexer/v1/task/{taskId}` - -Retrieve the 'payload' of a task. - -`GET /druid/indexer/v1/task/{taskId}/status` - -Retrieve the status of a task. - -`GET /druid/indexer/v1/task/{taskId}/segments` - -> This API is deprecated and will be removed in future releases. - -Retrieve information about the segments of a task. - -`GET /druid/indexer/v1/task/{taskId}/reports` - -Retrieve a [task completion report](../ingestion/tasks.md#task-reports) for a task. Only works for completed tasks. - -`POST /druid/indexer/v1/task` - -Endpoint for submitting tasks and supervisor specs to the Overlord. Returns the taskId of the submitted task. - -`POST /druid/indexer/v1/task/{taskId}/shutdown` - -Shuts down a task. - -`POST /druid/indexer/v1/datasources/{dataSource}/shutdownAllTasks` - -Shuts down all tasks for a dataSource. - -`POST /druid/indexer/v1/taskStatus` - -Retrieve list of task status objects for list of task id strings in request body. - -`DELETE /druid/indexer/v1/pendingSegments/{dataSource}` - -Manually clean up pending segments table in metadata storage for `datasource`. Returns a JSON object response with -`numDeleted` and count of rows deleted from the pending segments table. This API is used by the -`druid.coordinator.kill.pendingSegments.on` [coordinator setting](../configuration/index.md#coordinator-operation) -which automates this operation to perform periodically. - -#### Supervisors - -`GET /druid/indexer/v1/supervisor` - -Returns a list of strings of the currently active supervisor ids. - -`GET /druid/indexer/v1/supervisor?full` - -Returns a list of objects of the currently active supervisors. - -|Field|Type|Description| -|---|---|---| -|`id`|String|supervisor unique identifier| -|`state`|String|basic state of the supervisor. Available states:`UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. Check [Kafka Docs](../development/extensions-core/kafka-supervisor-operations.md) for details.| -|`detailedState`|String|supervisor specific state. See documentation of specific supervisor for details: [Kafka](../development/extensions-core/kafka-ingestion.md) or [Kinesis](../development/extensions-core/kinesis-ingestion.md)| -|`healthy`|Boolean|true or false indicator of overall supervisor health| -|`spec`|SupervisorSpec|JSON specification of supervisor| - -`GET /druid/indexer/v1/supervisor?state=true` - -Returns a list of objects of the currently active supervisors and their current state. - -|Field|Type|Description| -|---|---|---| -|`id`|String|supervisor unique identifier| -|`state`|String|basic state of the supervisor. Available states: `UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. Check [Kafka Docs](../development/extensions-core/kafka-supervisor-operations.md) for details.| -|`detailedState`|String|supervisor specific state. See documentation of the specific supervisor for details: [Kafka](../development/extensions-core/kafka-ingestion.md) or [Kinesis](../development/extensions-core/kinesis-ingestion.md)| -|`healthy`|Boolean|true or false indicator of overall supervisor health| -|`suspended`|Boolean|true or false indicator of whether the supervisor is in suspended state| - -`GET /druid/indexer/v1/supervisor/` - -Returns the current spec for the supervisor with the provided ID. - -`GET /druid/indexer/v1/supervisor//status` - -Returns the current status of the supervisor with the provided ID. - -`GET/druid/indexer/v1/supervisor/history` - -Returns an audit history of specs for all supervisors (current and past). - -`GET /druid/indexer/v1/supervisor//history` - -Returns an audit history of specs for the supervisor with the provided ID. - -`POST /druid/indexer/v1/supervisor` - -Create a new supervisor or update an existing one. - -`POST /druid/indexer/v1/supervisor//suspend` - -Suspend the current running supervisor of the provided ID. Responds with updated SupervisorSpec. - -`POST /druid/indexer/v1/supervisor/suspendAll` - -Suspend all supervisors at once. - -`POST /druid/indexer/v1/supervisor//resume` - -Resume indexing tasks for a supervisor. Responds with updated SupervisorSpec. - -`POST /druid/indexer/v1/supervisor/resumeAll` - -Resume all supervisors at once. - -`POST /druid/indexer/v1/supervisor//reset` - -Reset the specified supervisor. - -`POST /druid/indexer/v1/supervisor//terminate` - -Terminate a supervisor of the provided ID. - -`POST /druid/indexer/v1/supervisor/terminateAll` - -Terminate all supervisors at once. - -`POST /druid/indexer/v1/supervisor//shutdown` - -> This API is deprecated and will be removed in future releases. -> Please use the equivalent `terminate` instead. - -Shutdown a supervisor. - -#### Dynamic configuration - -See [Overlord Dynamic Configuration](../configuration/index.md#overlord-dynamic-configuration) for details. - -Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` -as in `2016-06-27_2016-06-28`. - -`GET /druid/indexer/v1/worker` - -Retrieves current overlord dynamic configuration. - -`GET /druid/indexer/v1/worker/history?interval={interval}&count={count}` - -Retrieves history of changes to overlord dynamic configuration. Accepts `interval` and `count` query string parameters -to filter by interval and limit the number of results respectively. - -`GET /druid/indexer/v1/workers` - -Retrieves a list of all the worker nodes in the cluster along with its metadata. - -`GET /druid/indexer/v1/scaling` - -Retrieves overlord scaling events if auto-scaling runners are in use. - -`POST /druid/indexer/v1/worker` - -Update overlord dynamic worker configuration. - -## Data server - -This section documents the API endpoints for the processes that reside on Data servers (MiddleManagers/Peons and Historicals) -in the suggested [three-server configuration](../design/processes.md#server-types). - -### MiddleManager - -`GET /druid/worker/v1/enabled` - -Check whether a MiddleManager is in an enabled or disabled state. Returns JSON object keyed by the combined `druid.host` -and `druid.port` with the boolean state as the value. - -```json -{"localhost:8091":true} -``` - -`GET /druid/worker/v1/tasks` - -Retrieve a list of active tasks being run on MiddleManager. Returns JSON list of taskid strings. Normal usage should -prefer to use the `/druid/indexer/v1/tasks` [Overlord API](#overlord) or one of it's task state specific variants instead. - -```json -["index_wikiticker_2019-02-11T02:20:15.316Z"] -``` - -`GET /druid/worker/v1/task/{taskid}/log` - -Retrieve task log output stream by task id. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/log` -[Overlord API](#overlord) instead. - -`POST /druid/worker/v1/disable` - -Disable a MiddleManager, causing it to stop accepting new tasks but complete all existing tasks. Returns JSON object -keyed by the combined `druid.host` and `druid.port`: - -```json -{"localhost:8091":"disabled"} -``` - -`POST /druid/worker/v1/enable` - -Enable a MiddleManager, allowing it to accept new tasks again if it was previously disabled. Returns JSON object -keyed by the combined `druid.host` and `druid.port`: - -```json -{"localhost:8091":"enabled"} -``` - -`POST /druid/worker/v1/task/{taskid}/shutdown` - -Shutdown a running task by `taskid`. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/shutdown` -[Overlord API](#overlord) instead. Returns JSON: - -```json -{"task":"index_kafka_wikiticker_f7011f8ffba384b_fpeclode"} -``` - - -### Peon - -`GET /druid/worker/v1/chat/{taskId}/rowStats` - -Retrieve a live row stats report from a Peon. See [task reports](../ingestion/tasks.md#task-reports) for more details. - -`GET /druid/worker/v1/chat/{taskId}/unparseableEvents` - -Retrieve an unparseable events report from a Peon. See [task reports](../ingestion/tasks.md#task-reports) for more details. - -### Historical - -#### Segment loading - -`GET /druid/historical/v1/loadstatus` - -Returns JSON of the form `{"cacheInitialized":}`, where value is either `true` or `false` indicating if all -segments in the local cache have been loaded. This can be used to know when a Historical process is ready -to be queried after a restart. - -`GET /druid/historical/v1/readiness` - -Similar to `/druid/historical/v1/loadstatus`, but instead of returning JSON with a flag, responses 200 OK if segments -in the local cache have been loaded, and 503 SERVICE UNAVAILABLE, if they haven't. - - -## Query server - -This section documents the API endpoints for the processes that reside on Query servers (Brokers) in the suggested [three-server configuration](../design/processes.md#server-types). - -### Broker - -#### Datasource information - -Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` -as in `2016-06-27_2016-06-28`. - -> Note: Much of this information is available in a simpler, easier-to-use form through the Druid SQL -> [`INFORMATION_SCHEMA.TABLES`](../querying/sql-metadata-tables.md#tables-table), -> [`INFORMATION_SCHEMA.COLUMNS`](../querying/sql-metadata-tables.md#columns-table), and -> [`sys.segments`](../querying/sql-metadata-tables.md#segments-table) tables. - -`GET /druid/v2/datasources` - -Returns a list of queryable datasources. - -`GET /druid/v2/datasources/{dataSourceName}` - -Returns the dimensions and metrics of the datasource. Optionally, you can provide request parameter "full" to get list of served intervals with dimensions and metrics being served for those intervals. You can also provide request param "interval" explicitly to refer to a particular interval. - -If no interval is specified, a default interval spanning a configurable period before the current time will be used. The default duration of this interval is specified in ISO 8601 duration format via: `druid.query.segmentMetadata.defaultHistory` - -`GET /druid/v2/datasources/{dataSourceName}/dimensions` - -> This API is deprecated and will be removed in future releases. Please use [SegmentMetadataQuery](../querying/segmentmetadataquery.md) instead -> which provides more comprehensive information and supports all dataSource types including streaming dataSources. It's also encouraged to use [INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md) -> if you're using SQL. -> -Returns the dimensions of the datasource. - -`GET /druid/v2/datasources/{dataSourceName}/metrics` - -> This API is deprecated and will be removed in future releases. Please use [SegmentMetadataQuery](../querying/segmentmetadataquery.md) instead -> which provides more comprehensive information and supports all dataSource types including streaming dataSources. It's also encouraged to use [INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md) -> if you're using SQL. - -Returns the metrics of the datasource. - -`GET /druid/v2/datasources/{dataSourceName}/candidates?intervals={comma-separated-intervals}&numCandidates={numCandidates}` - -Returns segment information lists including server locations for the given datasource and intervals. If "numCandidates" is not specified, it will return all servers for each interval. - -#### Load Status - -`GET /druid/broker/v1/loadstatus` - -Returns a flag indicating if the Broker knows about all segments in the cluster. This can be used to know when a Broker process is ready to be queried after a restart. - -`GET /druid/broker/v1/readiness` - -Similar to `/druid/broker/v1/loadstatus`, but instead of returning a JSON, responses 200 OK if its ready and otherwise 503 SERVICE UNAVAILABLE. - -#### Queries - -`POST /druid/v2/` - -The endpoint for submitting queries. Accepts an option `?pretty` that pretty prints the results. - -`POST /druid/v2/candidates/` - -Returns segment information lists including server locations for the given query.. - -### Router - -> Note: Much of this information is available in a simpler, easier-to-use form through the Druid SQL -> [`INFORMATION_SCHEMA.TABLES`](../querying/sql-metadata-tables.md#tables-table), -> [`INFORMATION_SCHEMA.COLUMNS`](../querying/sql-metadata-tables.md#columns-table), and -> [`sys.segments`](../querying/sql-metadata-tables.md#segments-table) tables. - -`GET /druid/v2/datasources` - -Returns a list of queryable datasources. - -`GET /druid/v2/datasources/{dataSourceName}` - -Returns the dimensions and metrics of the datasource. - -`GET /druid/v2/datasources/{dataSourceName}/dimensions` - -Returns the dimensions of the datasource. - -`GET /druid/v2/datasources/{dataSourceName}/metrics` - -Returns the metrics of the datasource. +This topic is an index to the Apache Druid API documentation. + +## HTTP APIs +* [Druid SQL queries](./sql-api.md) to submit SQL queries using the Druid SQL API. +* [SQL-based ingestion](./sql-ingestion-api.md) to submit SQL-based batch ingestion requests. +* [JSON querying](./json-querying-api.md) to submit JSON-based native queries. +* [Tasks](./tasks-api.md) to manage data ingestion operations. +* [Supervisors](./supervisor-api.md) to manage supervisors for data ingestion lifecycle and data processing. +* [Retention rules](./retention-rules-api.md) to define and manage data retention rules across datasources. +* [Data management](./data-management-api.md) to manage data segments. +* [Automatic compaction](./automatic-compaction-api.md) to optimize segment sizes after ingestion. +* [Lookups](./lookups-api.md) to manage and modify key-value datasources. +* [Service status](./service-status-api.md) to monitor components within the Druid cluster. +* [Dynamic configuration](./dynamic-configuration-api.md) to configure the behavior of the Coordinator and Overlord processes. +* [Legacy metadata](./legacy-metadata-api.md) to retrieve datasource metadata. + +## Java APIs +* [SQL JDBC driver](./sql-jdbc.md) to connect to Druid and make Druid SQL queries using the Avatica JDBC driver. \ No newline at end of file diff --git a/docs/api-reference/automatic-compaction-api.md b/docs/api-reference/automatic-compaction-api.md new file mode 100644 index 00000000000..ea0f824190a --- /dev/null +++ b/docs/api-reference/automatic-compaction-api.md @@ -0,0 +1,91 @@ +--- +id: automatic-compaction-api +title: Automatic compaction API +sidebar_label: Automatic compaction +--- + + + +This document describes status and configuration API endpoints for [automatic compaction](../data-management/automatic-compaction.md) in Apache Druid. + +## Automatic compaction status + +`GET /druid/coordinator/v1/compaction/progress?dataSource={dataSource}` + +Returns the total size of segments awaiting compaction for the given dataSource. The specified dataSource must have [automatic compaction](../data-management/automatic-compaction.md) enabled. + +`GET /druid/coordinator/v1/compaction/status` + +Returns the status and statistics from the auto-compaction run of all dataSources which have auto-compaction enabled in the latest run. The response payload includes a list of `latestStatus` objects. Each `latestStatus` represents the status for a dataSource (which has/had auto-compaction enabled). + +The `latestStatus` object has the following keys: +* `dataSource`: name of the datasource for this status information +* `scheduleStatus`: auto-compaction scheduling status. Possible values are `NOT_ENABLED` and `RUNNING`. Returns `RUNNING ` if the dataSource has an active auto-compaction config submitted. Otherwise, returns `NOT_ENABLED`. +* `bytesAwaitingCompaction`: total bytes of this datasource waiting to be compacted by the auto-compaction (only consider intervals/segments that are eligible for auto-compaction) +* `bytesCompacted`: total bytes of this datasource that are already compacted with the spec set in the auto-compaction config +* `bytesSkipped`: total bytes of this datasource that are skipped (not eligible for auto-compaction) by the auto-compaction +* `segmentCountAwaitingCompaction`: total number of segments of this datasource waiting to be compacted by the auto-compaction (only consider intervals/segments that are eligible for auto-compaction) +* `segmentCountCompacted`: total number of segments of this datasource that are already compacted with the spec set in the auto-compaction config +* `segmentCountSkipped`: total number of segments of this datasource that are skipped (not eligible for auto-compaction) by the auto-compaction +* `intervalCountAwaitingCompaction`: total number of intervals of this datasource waiting to be compacted by the auto-compaction (only consider intervals/segments that are eligible for auto-compaction) +* `intervalCountCompacted`: total number of intervals of this datasource that are already compacted with the spec set in the auto-compaction config +* `intervalCountSkipped`: total number of intervals of this datasource that are skipped (not eligible for auto-compaction) by the auto-compaction + +`GET /druid/coordinator/v1/compaction/status?dataSource={dataSource}` + +Similar to the API `/druid/coordinator/v1/compaction/status` above but filters response to only return information for the dataSource given. +The dataSource must have auto-compaction enabled. + +## Automatic compaction configuration + +`GET /druid/coordinator/v1/config/compaction` + +Returns all automatic compaction configs. + +`GET /druid/coordinator/v1/config/compaction/{dataSource}` + +Returns an automatic compaction config of a dataSource. + +`GET /druid/coordinator/v1/config/compaction/{dataSource}/history?interval={interval}&count={count}` + +Returns the history of the automatic compaction config for a dataSource. Optionally accepts `interval` and `count` +query string parameters to filter by interval and limit the number of results respectively. If the dataSource does not +exist or there is no compaction history for the dataSource, an empty list is returned. + +The response contains a list of objects with the following keys: +* `globalConfig`: A json object containing automatic compaction config that applies to the entire cluster. +* `compactionConfig`: A json object containing the automatic compaction config for the datasource. +* `auditInfo`: A json object that contains information about the change made - like `author`, `comment` and `ip`. +* `auditTime`: The date and time when the change was made. + +`POST /druid/coordinator/v1/config/compaction/taskslots?ratio={someRatio}&max={someMaxSlots}` + +Update the capacity for compaction tasks. `ratio` and `max` are used to limit the max number of compaction tasks. +They mean the ratio of the total task slots to the compaction task slots and the maximum number of task slots for compaction tasks, respectively. The actual max number of compaction tasks is `min(max, ratio * total task slots)`. +Note that `ratio` and `max` are optional and can be omitted. If they are omitted, default values (0.1 and unbounded) +will be set for them. + +`POST /druid/coordinator/v1/config/compaction` + +Creates or updates the [automatic compaction](../data-management/automatic-compaction.md) config for a dataSource. See [Automatic compaction dynamic configuration](../configuration/index.md#automatic-compaction-dynamic-configuration) for configuration details. + +`DELETE /druid/coordinator/v1/config/compaction/{dataSource}` + +Removes the automatic compaction config for a dataSource. diff --git a/docs/api-reference/data-management-api.md b/docs/api-reference/data-management-api.md new file mode 100644 index 00000000000..d3fd75ac698 --- /dev/null +++ b/docs/api-reference/data-management-api.md @@ -0,0 +1,79 @@ +--- +id: data-management-api +title: Data management API +sidebar_label: Data management +--- + + + +This document describes the data management API endpoints for Apache Druid. This includes information on how to mark segments as `used` or `unused` and delete them from Druid. + +## Note for Coordinator's POST and DELETE APIs + +While segments may be enabled by issuing POST requests for the datasources, the Coordinator may again disable segments if they match any configured [drop rules](../operations/rule-configuration.md#drop-rules). Even if segments are enabled by these APIs, you must configure a [load rule](../operations/rule-configuration.md#load-rules) to load them onto Historical processes. If an indexing or kill task runs at the same time these APIs are invoked, the behavior is undefined. Some segments might be killed and others might be enabled. It's also possible that all segments might be disabled, but the indexing task can still read data from those segments and succeed. + +> Avoid using indexing or kill tasks and these APIs at the same time for the same datasource and time chunk. + +`POST /druid/coordinator/v1/datasources/{dataSourceName}` + +Marks as used all segments belonging to a datasource. Returns a JSON object of the form +`{"numChangedSegments": }` with the number of segments in the database whose state has been changed (that is, +the segments were marked as used) as the result of this API call. + +`POST /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` + +Marks as used a segment of a datasource. Returns a JSON object of the form `{"segmentStateChanged": }` with +the boolean indicating if the state of the segment has been changed (that is, the segment was marked as used) as the +result of this API call. + +`POST /druid/coordinator/v1/datasources/{dataSourceName}/markUsed` + +`POST /druid/coordinator/v1/datasources/{dataSourceName}/markUnused` + +Marks segments (un)used for a datasource by interval or set of segment Ids. When marking used only segments that are not overshadowed will be updated. + +The request payload contains the interval or set of segment IDs to be marked unused. +Either interval or segment IDs should be provided, if both or none are provided in the payload, the API would throw an error (400 BAD REQUEST). + +Interval specifies the start and end times as IS0 8601 strings. `interval=(start/end)` where start and end both are inclusive and only the segments completely contained within the specified interval will be disabled, partially overlapping segments will not be affected. + +JSON Request Payload: + + |Key|Description|Example| +|----------|-------------|---------| +|`interval`|The interval for which to mark segments unused|`"2015-09-12T03:00:00.000Z/2015-09-12T05:00:00.000Z"`| +|`segmentIds`|Set of segment IDs to be marked unused|`["segmentId1", "segmentId2"]`| + +`DELETE /druid/coordinator/v1/datasources/{dataSourceName}` + +Marks as unused all segments belonging to a datasource. Returns a JSON object of the form +`{"numChangedSegments": }` with the number of segments in the database whose state has been changed (that is, +the segments were marked as unused) as the result of this API call. + +`DELETE /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}` +`@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myInterval}` + +Runs a [Kill task](../ingestion/tasks.md) for a given interval and datasource. + +`DELETE /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` + +Marks as unused a segment of a datasource. Returns a JSON object of the form `{"segmentStateChanged": }` with +the boolean indicating if the state of the segment has been changed (that is, the segment was marked as unused) as the +result of this API call. \ No newline at end of file diff --git a/docs/api-reference/dynamic-configuration-api.md b/docs/api-reference/dynamic-configuration-api.md new file mode 100644 index 00000000000..551c05300b8 --- /dev/null +++ b/docs/api-reference/dynamic-configuration-api.md @@ -0,0 +1,75 @@ +--- +id: dynamic-configuration-api +title: Dynamic configuration API +sidebar_label: Dynamic configuration +--- + + + +This document describes the API endpoints to retrieve and manage the dynamic configurations for the [Coordinator](../configuration/index.html#overlord-dynamic-configuration) and [Overlord](../configuration/index.html#dynamic-configuration) in Apache Druid. + +## Coordinator dynamic configuration + +See [Coordinator Dynamic Configuration](../configuration/index.md#dynamic-configuration) for details. + +Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` +as in `2016-06-27_2016-06-28`. + +`GET /druid/coordinator/v1/config` + +Retrieves current coordinator dynamic configuration. + +`GET /druid/coordinator/v1/config/history?interval={interval}&count={count}` + +Retrieves history of changes to overlord dynamic configuration. Accepts `interval` and `count` query string parameters +to filter by interval and limit the number of results respectively. + +`POST /druid/coordinator/v1/config` + +Update overlord dynamic worker configuration. + + +## Overlord dynamic configuration + +See [Overlord Dynamic Configuration](../configuration/index.md#overlord-dynamic-configuration) for details. + +Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` +as in `2016-06-27_2016-06-28`. + +`GET /druid/indexer/v1/worker` + +Retrieves current overlord dynamic configuration. + +`GET /druid/indexer/v1/worker/history?interval={interval}&count={count}` + +Retrieves history of changes to overlord dynamic configuration. Accepts `interval` and `count` query string parameters +to filter by interval and limit the number of results respectively. + +`GET /druid/indexer/v1/workers` + +Retrieves a list of all the worker nodes in the cluster along with its metadata. + +`GET /druid/indexer/v1/scaling` + +Retrieves overlord scaling events if auto-scaling runners are in use. + +`POST /druid/indexer/v1/worker` + +Update overlord dynamic worker configuration. \ No newline at end of file diff --git a/docs/api-reference/json-querying-api.md b/docs/api-reference/json-querying-api.md new file mode 100644 index 00000000000..87a311705a8 --- /dev/null +++ b/docs/api-reference/json-querying-api.md @@ -0,0 +1,36 @@ +--- +id: json-querying-api +title: JSON querying API +sidebar_label: JSON querying +--- + + + +This document describes the API endpoints to submit JSON-based [native queries](../querying/querying.md) to Apache Druid. + +## Queries + +`POST /druid/v2/` + +The endpoint for submitting queries. Accepts an option `?pretty` that pretty prints the results. + +`POST /druid/v2/candidates/` + +Returns segment information lists including server locations for the given query. \ No newline at end of file diff --git a/docs/api-reference/legacy-metadata-api.md b/docs/api-reference/legacy-metadata-api.md new file mode 100644 index 00000000000..bbb4ebb324d --- /dev/null +++ b/docs/api-reference/legacy-metadata-api.md @@ -0,0 +1,315 @@ +--- +id: legacy-metadata-api +title: Legacy metadata API +sidebar_label: Legacy metadata +--- + + + +This document describes the legacy API endpoints to retrieve datasource metadata from Apache Druid. Use the [SQL metadata tables](/querying/sql-metadata-tables.md) to retrieve datasource metadata instead. + +## Segment loading + +`GET /druid/coordinator/v1/loadstatus` + +Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster. + +`GET /druid/coordinator/v1/loadstatus?simple` + +Returns the number of segments left to load until segments that should be loaded in the cluster are available for queries. This does not include segment replication counts. + +`GET /druid/coordinator/v1/loadstatus?full` + +Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts. + +`GET /druid/coordinator/v1/loadstatus?full&computeUsingClusterView` + +Returns the number of segments not yet loaded for each tier until all segments loading in the cluster are available. +The result includes segment replication counts. It also factors in the number of available nodes that are of a service type that can load the segment when computing the number of segments remaining to load. +A segment is considered fully loaded when: +- Druid has replicated it the number of times configured in the corresponding load rule. +- Or the number of replicas for the segment in each tier where it is configured to be replicated equals the available nodes of a service type that are currently allowed to load the segment in the tier. + +`GET /druid/coordinator/v1/loadqueue` + +Returns the ids of segments to load and drop for each Historical process. + +`GET /druid/coordinator/v1/loadqueue?simple` + +Returns the number of segments to load and drop, as well as the total segment load and drop size in bytes for each Historical process. + +`GET /druid/coordinator/v1/loadqueue?full` + +Returns the serialized JSON of segments to load and drop for each Historical process. + +## Segment loading by datasource + +Note that all _interval_ query parameters are ISO 8601 strings—for example, 2016-06-27/2016-06-28. +Also note that these APIs only guarantees that the segments are available at the time of the call. +Segments can still become missing because of historical process failures or any other reasons afterward. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?forceMetadataRefresh={boolean}&interval={myInterval}` + +Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster for the given +datasource over the given interval (or last 2 weeks if interval is not given). `forceMetadataRefresh` is required to be set. +* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store +(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms +of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status) +* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. +If no used segments are found for the given inputs, this API returns `204 No Content` + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?simple&forceMetadataRefresh={boolean}&interval={myInterval}` + +Returns the number of segments left to load until segments that should be loaded in the cluster are available for the given datasource +over the given interval (or last 2 weeks if interval is not given). This does not include segment replication counts. `forceMetadataRefresh` is required to be set. +* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store +(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms +of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status) +* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. +If no used segments are found for the given inputs, this API returns `204 No Content` + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?full&forceMetadataRefresh={boolean}&interval={myInterval}` + +Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available for the given datasource over the given interval (or last 2 weeks if interval is not given). This includes segment replication counts. `forceMetadataRefresh` is required to be set. +* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store +(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms +of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status) +* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. + +You can pass the optional query parameter `computeUsingClusterView` to factor in the available cluster services when calculating +the segments left to load. See [Coordinator Segment Loading](#segment-loading) for details. +If no used segments are found for the given inputs, this API returns `204 No Content` + +## Metadata store information + +> Note: Much of this information is available in a simpler, easier-to-use form through the Druid SQL +> [`sys.segments`](../querying/sql-metadata-tables.md#segments-table) table. + +`GET /druid/coordinator/v1/metadata/segments` + +Returns a list of all segments for each datasource enabled in the cluster. + +`GET /druid/coordinator/v1/metadata/segments?datasources={dataSourceName1}&datasources={dataSourceName2}` + +Returns a list of all segments for one or more specific datasources enabled in the cluster. + +`GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus` + +Returns a list of all segments for each datasource with the full segment metadata and an extra field `overshadowed`. + +`GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&datasources={dataSourceName1}&datasources={dataSourceName2}` + +Returns a list of all segments for one or more specific datasources with the full segment metadata and an extra field `overshadowed`. + +`GET /druid/coordinator/v1/metadata/datasources` + +Returns a list of the names of datasources with at least one used segment in the cluster, retrieved from the metadata database. Users should call this API to get the eventual state that the system will be in. + +`GET /druid/coordinator/v1/metadata/datasources?includeUnused` + +Returns a list of the names of datasources, regardless of whether there are used segments belonging to those datasources in the cluster or not. + +`GET /druid/coordinator/v1/metadata/datasources?includeDisabled` + +Returns a list of the names of datasources, regardless of whether the datasource is disabled or not. + +`GET /druid/coordinator/v1/metadata/datasources?full` + +Returns a list of all datasources with at least one used segment in the cluster. Returns all metadata about those datasources as stored in the metadata store. + +`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}` + +Returns full metadata for a datasource as stored in the metadata store. + +`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments` + +Returns a list of all segments for a datasource as stored in the metadata store. + +`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full` + +Returns a list of all segments for a datasource with the full segment metadata as stored in the metadata store. + +`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments/{segmentId}` + +Returns full segment metadata for a specific segment as stored in the metadata store, if the segment is used. If the +segment is unused, or is unknown, a 404 response is returned. + +`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments` + +Returns a list of all segments, overlapping with any of given intervals, for a datasource as stored in the metadata store. Request body is array of string IS0 8601 intervals like `[interval1, interval2,...]`—for example, `["2012-01-01T00:00:00.000/2012-01-03T00:00:00.000", "2012-01-05T00:00:00.000/2012-01-07T00:00:00.000"]`. + +`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full` + +Returns a list of all segments, overlapping with any of given intervals, for a datasource with the full segment metadata as stored in the metadata store. Request body is array of string ISO 8601 intervals like `[interval1, interval2,...]`—for example, `["2012-01-01T00:00:00.000/2012-01-03T00:00:00.000", "2012-01-05T00:00:00.000/2012-01-07T00:00:00.000"]`. + + + +## Datasources + +Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`—for example, `2016-06-27_2016-06-28`. + +`GET /druid/coordinator/v1/datasources` + +Returns a list of datasource names found in the cluster as seen by the coordinator. This view is updated every [`druid.coordinator.period`](../configuration/index.md#coordinator-operation). + +`GET /druid/coordinator/v1/datasources?simple` + +Returns a list of JSON objects containing the name and properties of datasources found in the cluster. Properties include segment count, total segment byte size, replicated total segment byte size, minTime, and maxTime. + +`GET /druid/coordinator/v1/datasources?full` + +Returns a list of datasource names found in the cluster with all metadata about those datasources. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}` + +Returns a JSON object containing the name and properties of a datasource. Properties include segment count, total segment byte size, replicated total segment byte size, minTime, and maxTime. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}?full` + +Returns full metadata for a datasource. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals` + +Returns a set of segment intervals. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals?simple` + +Returns a map of an interval to a JSON object containing the total byte size of segments and number of segments for that interval. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals?full` + +Returns a map of an interval to a map of segment metadata to a set of server names that contain the segment for that interval. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}` + +Returns a set of segment ids for an interval. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?simple` + +Returns a map of segment intervals contained within the specified interval to a JSON object containing the total byte size of segments and number of segments for an interval. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?full` + +Returns a map of segment intervals contained within the specified interval to a map of segment metadata to a set of server names that contain the segment for an interval. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}/serverview` + +Returns a map of segment intervals contained within the specified interval to information about the servers that contain the segment for an interval. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments` + +Returns a list of all segments for a datasource in the cluster. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments?full` + +Returns a list of all segments for a datasource in the cluster with the full segment metadata. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` + +Returns full segment metadata for a specific segment in the cluster. + +`GET /druid/coordinator/v1/datasources/{dataSourceName}/tiers` + +Return the tiers that a datasource exists in. + +## Intervals + +Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` as in `2016-06-27_2016-06-28`. + +`GET /druid/coordinator/v1/intervals` + +Returns all intervals for all datasources with total size and count. + +`GET /druid/coordinator/v1/intervals/{interval}` + +Returns aggregated total size and count for all intervals that intersect given ISO interval. + +`GET /druid/coordinator/v1/intervals/{interval}?simple` + +Returns total size and count for each interval within given ISO interval. + +`GET /druid/coordinator/v1/intervals/{interval}?full` + +Returns total size and count for each datasource for each interval within given ISO interval. + +## Server information + +`GET /druid/coordinator/v1/servers` + +Returns a list of servers URLs using the format `{hostname}:{port}`. Note that +processes that run with different types will appear multiple times with different +ports. + +`GET /druid/coordinator/v1/servers?simple` + +Returns a list of server data objects in which each object has the following keys: +* `host`: host URL include (`{hostname}:{port}`) +* `type`: process type (`indexer-executor`, `historical`) +* `currSize`: storage size currently used +* `maxSize`: maximum storage size +* `priority` +* `tier` + + +## Query server + +This section documents the API endpoints for the processes that reside on Query servers (Brokers) in the suggested [three-server configuration](../design/processes.md#server-types). + +### Broker + +#### Datasource information + +Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` +as in `2016-06-27_2016-06-28`. + +> Note: Much of this information is available in a simpler, easier-to-use form through the Druid SQL +> [`INFORMATION_SCHEMA.TABLES`](../querying/sql-metadata-tables.md#tables-table), +> [`INFORMATION_SCHEMA.COLUMNS`](../querying/sql-metadata-tables.md#columns-table), and +> [`sys.segments`](../querying/sql-metadata-tables.md#segments-table) tables. + +`GET /druid/v2/datasources` + +Returns a list of queryable datasources. + +`GET /druid/v2/datasources/{dataSourceName}` + +Returns the dimensions and metrics of the datasource. Optionally, you can provide request parameter "full" to get list of served intervals with dimensions and metrics being served for those intervals. You can also provide request param "interval" explicitly to refer to a particular interval. + +If no interval is specified, a default interval spanning a configurable period before the current time will be used. The default duration of this interval is specified in ISO 8601 duration format via: `druid.query.segmentMetadata.defaultHistory` + +`GET /druid/v2/datasources/{dataSourceName}/dimensions` + +> This API is deprecated and will be removed in future releases. Please use [SegmentMetadataQuery](../querying/segmentmetadataquery.md) instead +> which provides more comprehensive information and supports all dataSource types including streaming dataSources. It's also encouraged to use [INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md) +> if you're using SQL. +> +Returns the dimensions of the datasource. + +`GET /druid/v2/datasources/{dataSourceName}/metrics` + +> This API is deprecated and will be removed in future releases. Please use [SegmentMetadataQuery](../querying/segmentmetadataquery.md) instead +> which provides more comprehensive information and supports all dataSource types including streaming dataSources. It's also encouraged to use [INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md) +> if you're using SQL. + +Returns the metrics of the datasource. + +`GET /druid/v2/datasources/{dataSourceName}/candidates?intervals={comma-separated-intervals}&numCandidates={numCandidates}` + +Returns segment information lists including server locations for the given datasource and intervals. If "numCandidates" is not specified, it will return all servers for each interval. \ No newline at end of file diff --git a/docs/api-reference/lookups-api.md b/docs/api-reference/lookups-api.md new file mode 100644 index 00000000000..9238ade01af --- /dev/null +++ b/docs/api-reference/lookups-api.md @@ -0,0 +1,278 @@ +--- +id: lookups-api +title: Lookups API +sidebar_label: Lookups +--- + + + +This document describes the API endpoints to configure, update, retrieve, and manage lookups for Apache Druid. + +## Configure lookups + +### Bulk update +Lookups can be updated in bulk by posting a JSON object to `/druid/coordinator/v1/lookups/config`. The format of the json object is as follows: + +```json +{ + "": { + "": { + "version": "", + "lookupExtractorFactory": { + "type": "", + "": "" + } + } + } +} +``` + +Note that "version" is an arbitrary string assigned by the user, when making updates to existing lookup then user would need to specify a lexicographically higher version. + +For example, a config might look something like: + +```json +{ + "__default": { + "country_code": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "77483": "United States" + } + } + }, + "site_id": { + "version": "v0", + "lookupExtractorFactory": { + "type": "cachedNamespace", + "extractionNamespace": { + "type": "jdbc", + "connectorConfig": { + "createTables": true, + "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", + "user": "druid", + "password": "diurd" + }, + "table": "lookupTable", + "keyColumn": "country_id", + "valueColumn": "country_name", + "tsColumn": "timeColumn" + }, + "firstCacheTimeout": 120000, + "injective": true + } + }, + "site_id_customer1": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "847632": "Internal Use Only" + } + } + }, + "site_id_customer2": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "AHF77": "Home" + } + } + } + }, + "realtime_customer1": { + "country_code": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "77483": "United States" + } + } + }, + "site_id_customer1": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "847632": "Internal Use Only" + } + } + } + }, + "realtime_customer2": { + "country_code": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "77483": "United States" + } + } + }, + "site_id_customer2": { + "version": "v0", + "lookupExtractorFactory": { + "type": "map", + "map": { + "AHF77": "Home" + } + } + } + } +} +``` + +All entries in the map will UPDATE existing entries. No entries will be deleted. + +### Update lookup + +A `POST` to a particular lookup extractor factory via `/druid/coordinator/v1/lookups/config/{tier}/{id}` creates or updates that specific extractor factory. + +For example, a post to `/druid/coordinator/v1/lookups/config/realtime_customer1/site_id_customer1` might contain the following: + +```json +{ + "version": "v1", + "lookupExtractorFactory": { + "type": "map", + "map": { + "847632": "Internal Use Only" + } + } +} +``` + +This will replace the `site_id_customer1` lookup in the `realtime_customer1` with the definition above. + +Assign a unique version identifier each time you update a lookup extractor factory. Otherwise the call will fail. + +### Get all lookups + +A `GET` to `/druid/coordinator/v1/lookups/config/all` will return all known lookup specs for all tiers. + +### Get lookup + +A `GET` to a particular lookup extractor factory is accomplished via `/druid/coordinator/v1/lookups/config/{tier}/{id}` + +Using the prior example, a `GET` to `/druid/coordinator/v1/lookups/config/realtime_customer2/site_id_customer2` should return + +```json +{ + "version": "v1", + "lookupExtractorFactory": { + "type": "map", + "map": { + "AHF77": "Home" + } + } +} +``` + +### Delete lookup + +A `DELETE` to `/druid/coordinator/v1/lookups/config/{tier}/{id}` will remove that lookup from the cluster. If it was last lookup in the tier, then tier is deleted as well. + +### Delete tier + +A `DELETE` to `/druid/coordinator/v1/lookups/config/{tier}` will remove that tier from the cluster. + +### List tier names + +A `GET` to `/druid/coordinator/v1/lookups/config` will return a list of known tier names in the dynamic configuration. +To discover a list of tiers currently active in the cluster in addition to ones known in the dynamic configuration, the parameter `discover=true` can be added as per `/druid/coordinator/v1/lookups/config?discover=true`. + +### List lookup names + +A `GET` to `/druid/coordinator/v1/lookups/config/{tier}` will return a list of known lookup names for that tier. + +These end points can be used to get the propagation status of configured lookups to processes using lookups such as Historicals. + +## Lookup status + +### List load status of all lookups + +`GET /druid/coordinator/v1/lookups/status` with optional query parameter `detailed`. + +### List load status of lookups in a tier + +`GET /druid/coordinator/v1/lookups/status/{tier}` with optional query parameter `detailed`. + +### List load status of single lookup + +`GET /druid/coordinator/v1/lookups/status/{tier}/{lookup}` with optional query parameter `detailed`. + +### List lookup state of all processes + +`GET /druid/coordinator/v1/lookups/nodeStatus` with optional query parameter `discover` to discover tiers advertised by other Druid nodes, or by default, returning all configured lookup tiers. The default response will also include the lookups which are loaded, being loaded, or being dropped on each node, for each tier, including the complete lookup spec. Add the optional query parameter `detailed=false` to only include the 'version' of the lookup instead of the complete spec. + +### List lookup state of processes in a tier + +`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}` + +### List lookup state of single process + +`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}/{host:port}` + +## Internal API + +The Peon, Router, Broker, and Historical processes all have the ability to consume lookup configuration. +There is an internal API these processes use to list/load/drop their lookups starting at `/druid/listen/v1/lookups`. +These follow the same convention for return values as the cluster wide dynamic configuration. Following endpoints +can be used for debugging purposes but not otherwise. + +### Get lookups + +A `GET` to the process at `/druid/listen/v1/lookups` will return a json map of all the lookups currently active on the process. +The return value will be a json map of the lookups to their extractor factories. + +```json +{ + "site_id_customer2": { + "version": "v1", + "lookupExtractorFactory": { + "type": "map", + "map": { + "AHF77": "Home" + } + } + } +} +``` + +### Get lookup + +A `GET` to the process at `/druid/listen/v1/lookups/some_lookup_name` will return the LookupExtractorFactory for the lookup identified by `some_lookup_name`. +The return value will be the json representation of the factory. + +```json +{ + "version": "v1", + "lookupExtractorFactory": { + "type": "map", + "map": { + "AHF77": "Home" + } + } +} +``` \ No newline at end of file diff --git a/docs/api-reference/retention-rules-api.md b/docs/api-reference/retention-rules-api.md new file mode 100644 index 00000000000..42d5b8ecb29 --- /dev/null +++ b/docs/api-reference/retention-rules-api.md @@ -0,0 +1,69 @@ +--- +id: retention-rules-api +title: Retention rules API +sidebar_label: Retention rules +--- + + + +This document describes the API endpoints for managing retention rules in Apache Druid. + +## Retention rules + +Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` as in `2016-06-27_2016-06-28`. + +`GET /druid/coordinator/v1/rules` + +Returns all rules as JSON objects for all datasources in the cluster including the default datasource. + +`GET /druid/coordinator/v1/rules/{dataSourceName}` + +Returns all rules for a specified datasource. + +`GET /druid/coordinator/v1/rules/{dataSourceName}?full` + +Returns all rules for a specified datasource and includes default datasource. + +`GET /druid/coordinator/v1/rules/history?interval=` + +Returns audit history of rules for all datasources. Default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Coordinator `runtime.properties`. + +`GET /druid/coordinator/v1/rules/history?count=` + +Returns last `n` entries of audit history of rules for all datasources. + +`GET /druid/coordinator/v1/rules/{dataSourceName}/history?interval=` + +Returns audit history of rules for a specified datasource. Default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Coordinator `runtime.properties`. + +`GET /druid/coordinator/v1/rules/{dataSourceName}/history?count=` + +Returns last `n` entries of audit history of rules for a specified datasource. + +`POST /druid/coordinator/v1/rules/{dataSourceName}` + +POST with a list of rules in JSON form to update rules. + +Optional Header Parameters for auditing the config change can also be specified. + +|Header Param Name| Description | Default | +|----------|-------------|---------| +|`X-Druid-Author`| Author making the config change|`""`| +|`X-Druid-Comment`| Comment describing the change being done|`""`| diff --git a/docs/api-reference/service-status-api.md b/docs/api-reference/service-status-api.md new file mode 100644 index 00000000000..614d216ca0d --- /dev/null +++ b/docs/api-reference/service-status-api.md @@ -0,0 +1,176 @@ +--- +id: service-status-api +title: Service status API +sidebar_label: Service status +--- + + + +This document describes the API endpoints to retrieve service (process) status, cluster information for Apache Druid + +## Common + +All processes support the following endpoints. + +### Process information + +`GET /status` + +Returns the Druid version, loaded extensions, memory used, total memory, and other useful information about the process. + +`GET /status/health` + +Always returns a boolean `true` value with a 200 OK response, useful for automated health checks. + +`GET /status/properties` + +Returns the current configuration properties of the process. + +`GET /status/selfDiscovered/status` + +Returns a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has received a confirmation +from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the +cluster. It is recommended to not consider a Druid node "healthy" or "ready" in automated deployment/container +management systems until it returns `{"selfDiscovered": true}` from this endpoint. This is because a node may be +isolated from the rest of the cluster due to network issues and it doesn't make sense to consider nodes "healthy" in +this case. Also, when nodes such as Brokers use ZooKeeper segment discovery for building their view of the Druid cluster +(as opposed to HTTP segment discovery), they may be unusable until the ZooKeeper client is fully initialized and starts +to receive data from the ZooKeeper cluster. `{"selfDiscovered": true}` is a proxy event indicating that the ZooKeeper +client on the node has started to receive data from the ZooKeeper cluster and it's expected that all segments and other +nodes will be discovered by this node timely from this point. + +`GET /status/selfDiscovered` + +Similar to `/status/selfDiscovered/status`, but returns 200 OK response with empty body if the node has discovered itself +and 503 SERVICE UNAVAILABLE if the node hasn't discovered itself yet. This endpoint might be useful because some +monitoring checks such as AWS load balancer health checks are not able to look at the response body. + +## Master server + +### Coordinator + +#### Leadership + +`GET /druid/coordinator/v1/leader` + +Returns the current leader Coordinator of the cluster. + +`GET /druid/coordinator/v1/isLeader` + +Returns a JSON object with `leader` parameter, either true or false, indicating if this server is the current leader +Coordinator of the cluster. In addition, returns HTTP 200 if the server is the current leader and HTTP 404 if not. +This is suitable for use as a load balancer status check if you only want the active leader to be considered in-service +at the load balancer. + + + +### Overlord + +#### Leadership + +`GET /druid/indexer/v1/leader` + +Returns the current leader Overlord of the cluster. If you have multiple Overlords, just one is leading at any given time. The others are on standby. + +`GET /druid/indexer/v1/isLeader` + +This returns a JSON object with field `leader`, either true or false. In addition, this call returns HTTP 200 if the +server is the current leader and HTTP 404 if not. This is suitable for use as a load balancer status check if you +only want the active leader to be considered in-service at the load balancer. + +## Data server + +### MiddleManager + +`GET /druid/worker/v1/enabled` + +Check whether a MiddleManager is in an enabled or disabled state. Returns JSON object keyed by the combined `druid.host` +and `druid.port` with the boolean state as the value. + +```json +{"localhost:8091":true} +``` + +`GET /druid/worker/v1/tasks` + +Retrieve a list of active tasks being run on MiddleManager. Returns JSON list of taskid strings. Normal usage should +prefer to use the `/druid/indexer/v1/tasks` [Tasks API](./tasks-api.md) or one of it's task state specific variants instead. + +```json +["index_wikiticker_2019-02-11T02:20:15.316Z"] +``` + +`GET /druid/worker/v1/task/{taskid}/log` + +Retrieve task log output stream by task id. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/log` +[Tasks API](./tasks-api.md) instead. + +`POST /druid/worker/v1/disable` + +Disable a MiddleManager, causing it to stop accepting new tasks but complete all existing tasks. Returns JSON object +keyed by the combined `druid.host` and `druid.port`: + +```json +{"localhost:8091":"disabled"} +``` + +`POST /druid/worker/v1/enable` + +Enable a MiddleManager, allowing it to accept new tasks again if it was previously disabled. Returns JSON object +keyed by the combined `druid.host` and `druid.port`: + +```json +{"localhost:8091":"enabled"} +``` + +`POST /druid/worker/v1/task/{taskid}/shutdown` + +Shutdown a running task by `taskid`. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/shutdown` +[Tasks API](./tasks-api.md) instead. Returns JSON: + +```json +{"task":"index_kafka_wikiticker_f7011f8ffba384b_fpeclode"} +``` + + +## Historical +### Segment loading + +`GET /druid/historical/v1/loadstatus` + +Returns JSON of the form `{"cacheInitialized":}`, where value is either `true` or `false` indicating if all +segments in the local cache have been loaded. This can be used to know when a Historical process is ready +to be queried after a restart. + +`GET /druid/historical/v1/readiness` + +Similar to `/druid/historical/v1/loadstatus`, but instead of returning JSON with a flag, responses 200 OK if segments +in the local cache have been loaded, and 503 SERVICE UNAVAILABLE, if they haven't. + + +## Load Status + +`GET /druid/broker/v1/loadstatus` + +Returns a flag indicating if the Broker knows about all segments in the cluster. This can be used to know when a Broker process is ready to be queried after a restart. + +`GET /druid/broker/v1/readiness` + +Similar to `/druid/broker/v1/loadstatus`, but instead of returning a JSON, responses 200 OK if its ready and otherwise 503 SERVICE UNAVAILABLE. diff --git a/docs/api-reference/sql-api.md b/docs/api-reference/sql-api.md index 54cc3042d06..aaaf499851d 100644 --- a/docs/api-reference/sql-api.md +++ b/docs/api-reference/sql-api.md @@ -186,4 +186,4 @@ Druid returns an HTTP 404 response in the following cases: - `sqlQueryId` is incorrect. - The query completes before your cancellation request is processed. -Druid returns an HTTP 403 response for authorization failure. +Druid returns an HTTP 403 response for authorization failure. \ No newline at end of file diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md index 43acca9b49c..3050f595e67 100644 --- a/docs/api-reference/sql-ingestion-api.md +++ b/docs/api-reference/sql-ingestion-api.md @@ -123,7 +123,7 @@ print(response.text) | Field | Description | |---|---| -| `taskId` | Controller task ID. You can use Druid's standard [task APIs](api-reference.md#overlord) to interact with this controller task. | +| `taskId` | Controller task ID. You can use Druid's standard [Tasks API](./tasks-api.md) to interact with this controller task. | | `state` | Initial state for the query, which is "RUNNING". | ## Get the status for a query task @@ -628,4 +628,4 @@ print(response.text) { "task": "query-655efe33-781a-4c50-ae84-c2911b42d63c" } -``` +``` \ No newline at end of file diff --git a/docs/api-reference/supervisor-api.md b/docs/api-reference/supervisor-api.md new file mode 100644 index 00000000000..45e1bcddaea --- /dev/null +++ b/docs/api-reference/supervisor-api.md @@ -0,0 +1,111 @@ +--- +id: supervisor-api +title: Supervisor API +sidebar_label: Supervisors +--- + + + +This document describes the API endpoints to manage and monitor supervisors for Apache Druid. + +## Supervisors + +`GET /druid/indexer/v1/supervisor` + +Returns a list of strings of the currently active supervisor ids. + +`GET /druid/indexer/v1/supervisor?full` + +Returns a list of objects of the currently active supervisors. + +|Field|Type|Description| +|---|---|---| +|`id`|String|supervisor unique identifier| +|`state`|String|basic state of the supervisor. Available states:`UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. Check [Kafka Docs](../development/extensions-core/kafka-supervisor-operations.md) for details.| +|`detailedState`|String|supervisor specific state. See documentation of specific supervisor for details: [Kafka](../development/extensions-core/kafka-ingestion.md) or [Kinesis](../development/extensions-core/kinesis-ingestion.md)| +|`healthy`|Boolean|true or false indicator of overall supervisor health| +|`spec`|SupervisorSpec|JSON specification of supervisor| + +`GET /druid/indexer/v1/supervisor?state=true` + +Returns a list of objects of the currently active supervisors and their current state. + +|Field|Type|Description| +|---|---|---| +|`id`|String|supervisor unique identifier| +|`state`|String|basic state of the supervisor. Available states: `UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. Check [Kafka Docs](../development/extensions-core/kafka-supervisor-operations.md) for details.| +|`detailedState`|String|supervisor specific state. See documentation of the specific supervisor for details: [Kafka](../development/extensions-core/kafka-ingestion.md) or [Kinesis](../development/extensions-core/kinesis-ingestion.md)| +|`healthy`|Boolean|true or false indicator of overall supervisor health| +|`suspended`|Boolean|true or false indicator of whether the supervisor is in suspended state| + +`GET /druid/indexer/v1/supervisor/` + +Returns the current spec for the supervisor with the provided ID. + +`GET /druid/indexer/v1/supervisor//status` + +Returns the current status of the supervisor with the provided ID. + +`GET/druid/indexer/v1/supervisor/history` + +Returns an audit history of specs for all supervisors (current and past). + +`GET /druid/indexer/v1/supervisor//history` + +Returns an audit history of specs for the supervisor with the provided ID. + +`POST /druid/indexer/v1/supervisor` + +Create a new supervisor or update an existing one. + +`POST /druid/indexer/v1/supervisor//suspend` + +Suspend the current running supervisor of the provided ID. Responds with updated SupervisorSpec. + +`POST /druid/indexer/v1/supervisor/suspendAll` + +Suspend all supervisors at once. + +`POST /druid/indexer/v1/supervisor//resume` + +Resume indexing tasks for a supervisor. Responds with updated SupervisorSpec. + +`POST /druid/indexer/v1/supervisor/resumeAll` + +Resume all supervisors at once. + +`POST /druid/indexer/v1/supervisor//reset` + +Reset the specified supervisor. + +`POST /druid/indexer/v1/supervisor//terminate` + +Terminate a supervisor of the provided ID. + +`POST /druid/indexer/v1/supervisor/terminateAll` + +Terminate all supervisors at once. + +`POST /druid/indexer/v1/supervisor//shutdown` + +> This API is deprecated and will be removed in future releases. +> Please use the equivalent `terminate` instead. + +Shutdown a supervisor. \ No newline at end of file diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md new file mode 100644 index 00000000000..f8c1ca2e358 --- /dev/null +++ b/docs/api-reference/tasks-api.md @@ -0,0 +1,101 @@ +--- +id: tasks-api +title: Tasks API +sidebar_label: Tasks +--- + + + +This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. + +## Tasks + +Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` +as in `2016-06-27_2016-06-28`. + +`GET /druid/indexer/v1/tasks` + +Retrieve list of tasks. Accepts query string parameters `state`, `datasource`, `createdTimeInterval`, `max`, and `type`. + +|Query Parameter |Description | +|---|---| +|`state`|filter list of tasks by task state, valid options are `running`, `complete`, `waiting`, and `pending`.| +| `datasource`| return tasks filtered by Druid datasource.| +| `createdTimeInterval`| return tasks created within the specified interval. | +| `max`| maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `type`| filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| + + +`GET /druid/indexer/v1/completeTasks` + +Retrieve list of complete tasks. Equivalent to `/druid/indexer/v1/tasks?state=complete`. + +`GET /druid/indexer/v1/runningTasks` + +Retrieve list of running tasks. Equivalent to `/druid/indexer/v1/tasks?state=running`. + +`GET /druid/indexer/v1/waitingTasks` + +Retrieve list of waiting tasks. Equivalent to `/druid/indexer/v1/tasks?state=waiting`. + +`GET /druid/indexer/v1/pendingTasks` + +Retrieve list of pending tasks. Equivalent to `/druid/indexer/v1/tasks?state=pending`. + +`GET /druid/indexer/v1/task/{taskId}` + +Retrieve the 'payload' of a task. + +`GET /druid/indexer/v1/task/{taskId}/status` + +Retrieve the status of a task. + +`GET /druid/indexer/v1/task/{taskId}/segments` + +> This API is deprecated and will be removed in future releases. + +Retrieve information about the segments of a task. + +`GET /druid/indexer/v1/task/{taskId}/reports` + +Retrieve a [task completion report](../ingestion/tasks.md#task-reports) for a task. Only works for completed tasks. + +`POST /druid/indexer/v1/task` + +Endpoint for submitting tasks and supervisor specs to the Overlord. Returns the taskId of the submitted task. + +`POST /druid/indexer/v1/task/{taskId}/shutdown` + +Shuts down a task. + +`POST /druid/indexer/v1/datasources/{dataSource}/shutdownAllTasks` + +Shuts down all tasks for a dataSource. + +`POST /druid/indexer/v1/taskStatus` + +Retrieve list of task status objects for list of task id strings in request body. + +`DELETE /druid/indexer/v1/pendingSegments/{dataSource}` + +Manually clean up pending segments table in metadata storage for `datasource`. Returns a JSON object response with +`numDeleted` and count of rows deleted from the pending segments table. This API is used by the +`druid.coordinator.kill.pendingSegments.on` [coordinator setting](../configuration/index.md#coordinator-operation) +which automates this operation to perform periodically. diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 9ee061aa8e3..9698b28e431 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -999,7 +999,7 @@ These configuration options control Coordinator lookup management. See [dynamic ##### Automatic compaction dynamic configuration You can set or update [automatic compaction](../data-management/automatic-compaction.md) properties dynamically using the -[Coordinator API](../api-reference/api-reference.md#automatic-compaction-configuration) without restarting Coordinators. +[Automatic compaction API](../api-reference/automatic-compaction-api.md) without restarting Coordinators. For details about segment compaction, see [Segment size optimization](../operations/segment-optimization.md). diff --git a/docs/data-management/automatic-compaction.md b/docs/data-management/automatic-compaction.md index d7c0bb84957..05795339254 100644 --- a/docs/data-management/automatic-compaction.md +++ b/docs/data-management/automatic-compaction.md @@ -40,9 +40,9 @@ This topic guides you through setting up automatic compaction for your Druid clu ## Enable automatic compaction You can enable automatic compaction for a datasource using the web console or programmatically via an API. -This process differs for manual compaction tasks, which can be submitted from the [Tasks view of the web console](../operations/web-console.md) or the [Tasks API](../api-reference/api-reference.md#tasks). +This process differs for manual compaction tasks, which can be submitted from the [Tasks view of the web console](../operations/web-console.md) or the [Tasks API](../api-reference/tasks-api.md). -### web console +### Web console Use the web console to enable automatic compaction for a datasource as follows. @@ -59,10 +59,10 @@ To disable auto-compaction for a datasource, click **Delete** from the **Compact ### Compaction configuration API -Use the [Coordinator API](../api-reference/api-reference.md#automatic-compaction-status) to configure automatic compaction. +Use the [Automatic compaction API](../api-reference/automatic-compaction-api.md#automatic-compaction-status) to configure automatic compaction. To enable auto-compaction for a datasource, create a JSON object with the desired auto-compaction settings. See [Configure automatic compaction](#configure-automatic-compaction) for the syntax of an auto-compaction spec. -Send the JSON object as a payload in a [`POST` request](../api-reference/api-reference.md#automatic-compaction-configuration) to `/druid/coordinator/v1/config/compaction`. +Send the JSON object as a payload in a [`POST` request](../api-reference/automatic-compaction-api.md#automatic-compaction-configuration) to `/druid/coordinator/v1/config/compaction`. The following example configures auto-compaction for the `wikipedia` datasource: ```sh @@ -76,7 +76,7 @@ curl --location --request POST 'http://localhost:8081/druid/coordinator/v1/confi }' ``` -To disable auto-compaction for a datasource, send a [`DELETE` request](../api-reference/api-reference.md#automatic-compaction-configuration) to `/druid/coordinator/v1/config/compaction/{dataSource}`. Replace `{dataSource}` with the name of the datasource for which to disable auto-compaction. For example: +To disable auto-compaction for a datasource, send a [`DELETE` request](../api-reference/automatic-compaction-api.md#automatic-compaction-configuration) to `/druid/coordinator/v1/config/compaction/{dataSource}`. Replace `{dataSource}` with the name of the datasource for which to disable auto-compaction. For example: ```sh curl --location --request DELETE 'http://localhost:8081/druid/coordinator/v1/config/compaction/wikipedia' @@ -152,7 +152,7 @@ After the Coordinator has initiated auto-compaction, you can view compaction sta In the web console, the Datasources view displays auto-compaction statistics. The Tasks view shows the task information for compaction tasks that were triggered by the automatic compaction system. -To get statistics by API, send a [`GET` request](../api-reference/api-reference.md#automatic-compaction-status) to `/druid/coordinator/v1/compaction/status`. To filter the results to a particular datasource, pass the datasource name as a query parameter to the request—for example, `/druid/coordinator/v1/compaction/status?dataSource=wikipedia`. +To get statistics by API, send a [`GET` request](../api-reference/automatic-compaction-api.md#automatic-compaction-status) to `/druid/coordinator/v1/compaction/status`. To filter the results to a particular datasource, pass the datasource name as a query parameter to the request—for example, `/druid/coordinator/v1/compaction/status?dataSource=wikipedia`. ## Examples diff --git a/docs/data-management/delete.md b/docs/data-management/delete.md index ebabd69c4dc..60377d858ca 100644 --- a/docs/data-management/delete.md +++ b/docs/data-management/delete.md @@ -38,7 +38,7 @@ Deletion by time range happens in two steps: you have a backup. For documentation on disabling segments using the Coordinator API, see the -[Coordinator API reference](../api-reference/api-reference.md#coordinator-datasources). +[Legacy metadata API reference](../api-reference/legacy-metadata-api.md#datasources). A data deletion tutorial is available at [Tutorial: Deleting data](../tutorials/tutorial-delete-data.md). diff --git a/docs/design/broker.md b/docs/design/broker.md index 1c8c3be7b63..107048a7ada 100644 --- a/docs/design/broker.md +++ b/docs/design/broker.md @@ -31,7 +31,7 @@ For basic tuning guidance for the Broker process, see [Basic cluster tuning](../ ### HTTP endpoints -For a list of API endpoints supported by the Broker, see [Broker API](../api-reference/api-reference.md#broker). +For a list of API endpoints supported by the Broker, see [Broker API](../api-reference/legacy-metadata-api.md#broker). ### Overview diff --git a/docs/design/coordinator.md b/docs/design/coordinator.md index f0a162fe66c..8c235cd3d54 100644 --- a/docs/design/coordinator.md +++ b/docs/design/coordinator.md @@ -31,7 +31,7 @@ For basic tuning guidance for the Coordinator process, see [Basic cluster tuning ### HTTP endpoints -For a list of API endpoints supported by the Coordinator, see [Coordinator API](../api-reference/api-reference.md#coordinator). +For a list of API endpoints supported by the Coordinator, see [Service status API reference](../api-reference/service-status-api.md#coordinator). ### Overview @@ -92,7 +92,7 @@ Once some segments are found, it issues a [compaction task](../ingestion/tasks.m The maximum number of running compaction tasks is `min(sum of worker capacity * slotRatio, maxSlots)`. Note that even if `min(sum of worker capacity * slotRatio, maxSlots) = 0`, at least one compaction task is always submitted if the compaction is enabled for a dataSource. -See [Automatic compaction configuration API](../api-reference/api-reference.md#automatic-compaction-configuration) and [Automatic compaction configuration](../configuration/index.md#automatic-compaction-dynamic-configuration) to enable and configure automatic compaction. +See [Automatic compaction configuration API](../api-reference/automatic-compaction-api.md#automatic-compaction-configuration) and [Automatic compaction configuration](../configuration/index.md#automatic-compaction-dynamic-configuration) to enable and configure automatic compaction. Compaction tasks might fail due to the following reasons: diff --git a/docs/design/historical.md b/docs/design/historical.md index a2fb3032de3..4a276fd2708 100644 --- a/docs/design/historical.md +++ b/docs/design/historical.md @@ -31,7 +31,7 @@ For basic tuning guidance for the Historical process, see [Basic cluster tuning] ### HTTP endpoints -For a list of API endpoints supported by the Historical, please see the [API reference](../api-reference/api-reference.md#historical). +For a list of API endpoints supported by the Historical, please see the [Service status API reference](../api-reference/service-status-api.md#historical). ### Running diff --git a/docs/design/indexer.md b/docs/design/indexer.md index eedf0fc775e..9cce9c35059 100644 --- a/docs/design/indexer.md +++ b/docs/design/indexer.md @@ -35,7 +35,7 @@ For Apache Druid Indexer Process Configuration, see [Indexer Configuration](../c ### HTTP endpoints -The Indexer process shares the same HTTP endpoints as the [MiddleManager](../api-reference/api-reference.md#middlemanager). +The Indexer process shares the same HTTP endpoints as the [MiddleManager](../api-reference/service-status-api.md#middlemanager). ### Running diff --git a/docs/design/indexing-service.md b/docs/design/indexing-service.md index 793c31e81b0..537faa9a405 100644 --- a/docs/design/indexing-service.md +++ b/docs/design/indexing-service.md @@ -30,7 +30,7 @@ Indexing [tasks](../ingestion/tasks.md) are responsible for creating and [killin The indexing service is composed of three main components: [Peons](../design/peons.md) that can run a single task, [MiddleManagers](../design/middlemanager.md) that manage Peons, and an [Overlord](../design/overlord.md) that manages task distribution to MiddleManagers. Overlords and MiddleManagers may run on the same process or across multiple processes, while MiddleManagers and Peons always run on the same process. -Tasks are managed using API endpoints on the Overlord service. Please see [Overlord Task API](../api-reference/api-reference.md#tasks) for more information. +Tasks are managed using API endpoints on the Overlord service. Please see [Tasks API](../api-reference/tasks-api.md) for more information. ![Indexing Service](../assets/indexing_service.png "Indexing Service") diff --git a/docs/design/middlemanager.md b/docs/design/middlemanager.md index e0096c6b292..4feb32debae 100644 --- a/docs/design/middlemanager.md +++ b/docs/design/middlemanager.md @@ -31,7 +31,7 @@ For basic tuning guidance for the MiddleManager process, see [Basic cluster tuni ### HTTP endpoints -For a list of API endpoints supported by the MiddleManager, please see the [API reference](../api-reference/api-reference.md#middlemanager). +For a list of API endpoints supported by the MiddleManager, please see the [Service status API reference](../api-reference/service-status-api.md#middlemanager). ### Overview diff --git a/docs/design/overlord.md b/docs/design/overlord.md index 7c0ce9ce87a..ed0304a9324 100644 --- a/docs/design/overlord.md +++ b/docs/design/overlord.md @@ -31,7 +31,7 @@ For basic tuning guidance for the Overlord process, see [Basic cluster tuning](. ### HTTP endpoints -For a list of API endpoints supported by the Overlord, please see the [API reference](../api-reference/api-reference.md#overlord). +For a list of API endpoints supported by the Overlord, please see the [Service status API reference](../api-reference/service-status-api.md#overlord). ### Overview diff --git a/docs/design/peons.md b/docs/design/peons.md index d413dcb2503..e1348a25763 100644 --- a/docs/design/peons.md +++ b/docs/design/peons.md @@ -31,8 +31,6 @@ For basic tuning guidance for MiddleManager tasks, see [Basic cluster tuning](.. ### HTTP endpoints -For a list of API endpoints supported by the Peon, please see the [Peon API reference](../api-reference/api-reference.md#peon). - Peons run a single task in a single JVM. MiddleManager is responsible for creating Peons for running tasks. Peons should rarely (if ever for testing purposes) be run on their own. diff --git a/docs/design/router.md b/docs/design/router.md index 726f6831f16..d938c6a8eac 100644 --- a/docs/design/router.md +++ b/docs/design/router.md @@ -36,7 +36,7 @@ For basic tuning guidance for the Router process, see [Basic cluster tuning](../ ### HTTP endpoints -For a list of API endpoints supported by the Router, see [Router API](../api-reference/api-reference.md#router). +For a list of API endpoints supported by the Router, see [Legacy metadata API reference](../api-reference/legacy-metadata-api.md#datasource-information). ### Running diff --git a/docs/development/extensions-core/kafka-supervisor-operations.md b/docs/development/extensions-core/kafka-supervisor-operations.md index dbfa05174fb..e1de35eb2fd 100644 --- a/docs/development/extensions-core/kafka-supervisor-operations.md +++ b/docs/development/extensions-core/kafka-supervisor-operations.md @@ -25,7 +25,7 @@ description: "Reference topic for running and maintaining Apache Kafka superviso --> This topic contains operations reference information to run and maintain Apache Kafka supervisors for Apache Druid. It includes descriptions of how some supervisor APIs work within Kafka Indexing Service. -For all supervisor APIs, see [Supervisor APIs](../../api-reference/api-reference.md#supervisors). +For all supervisor APIs, see [Supervisor API reference](../../api-reference/supervisor-api.md). ## Getting Supervisor Status Report diff --git a/docs/development/extensions-core/kafka-supervisor-reference.md b/docs/development/extensions-core/kafka-supervisor-reference.md index 6ae3ad9102b..eff12d91850 100644 --- a/docs/development/extensions-core/kafka-supervisor-reference.md +++ b/docs/development/extensions-core/kafka-supervisor-reference.md @@ -205,7 +205,7 @@ The `tuningConfig` is optional and default parameters will be used if no `tuning | `indexSpecForIntermediatePersists`| | Defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. This can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [IndexSpec](#indexspec) for possible values. | no (default = same as `indexSpec`) | | `reportParseExceptions` | Boolean | *DEPRECATED*. If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped. Setting `reportParseExceptions` to true will override existing configurations for `maxParseExceptions` and `maxSavedParseExceptions`, setting `maxParseExceptions` to 0 and limiting `maxSavedParseExceptions` to no more than 1. | no (default == false) | | `handoffConditionTimeout` | Long | Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever. | no (default == 0) | -| `resetOffsetAutomatically` | Boolean | Controls behavior when Druid needs to read Kafka messages that are no longer available (i.e. when `OffsetOutOfRangeException` is encountered).

If false, the exception will bubble up, which will cause your tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation; potentially using the [Reset Supervisor API](../../api-reference/api-reference.md#supervisors). This mode is useful for production, since it will make you aware of issues with ingestion.

If true, Druid will automatically reset to the earlier or latest offset available in Kafka, based on the value of the `useEarliestOffset` property (earliest if true, latest if false). Note that this can lead to data being _DROPPED_ (if `useEarliestOffset` is false) or _DUPLICATED_ (if `useEarliestOffset` is true) without your knowledge. Messages will be logged indicating that a reset has occurred, but ingestion will continue. This mode is useful for non-production situations, since it will make Druid attempt to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.

This feature behaves similarly to the Kafka `auto.offset.reset` consumer property. | no (default == false) | +| `resetOffsetAutomatically` | Boolean | Controls behavior when Druid needs to read Kafka messages that are no longer available (i.e. when `OffsetOutOfRangeException` is encountered).

If false, the exception will bubble up, which will cause your tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation; potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it will make you aware of issues with ingestion.

If true, Druid will automatically reset to the earlier or latest offset available in Kafka, based on the value of the `useEarliestOffset` property (earliest if true, latest if false). Note that this can lead to data being _DROPPED_ (if `useEarliestOffset` is false) or _DUPLICATED_ (if `useEarliestOffset` is true) without your knowledge. Messages will be logged indicating that a reset has occurred, but ingestion will continue. This mode is useful for non-production situations, since it will make Druid attempt to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.

This feature behaves similarly to the Kafka `auto.offset.reset` consumer property. | no (default == false) | | `workerThreads` | Integer | The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation. | no (default == min(10, taskCount)) | | `chatAsync` | Boolean | If true, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If false, use synchronous communication in a thread pool of size `chatThreads`. | no (default == true) | | `chatThreads` | Integer | The number of threads that will be used for communicating with indexing tasks. Ignored if `chatAsync` is `true` (the default). | no (default == min(10, taskCount * replicas)) | diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md index 36e65f538c7..52abcba4b32 100644 --- a/docs/development/extensions-core/kinesis-ingestion.md +++ b/docs/development/extensions-core/kinesis-ingestion.md @@ -284,7 +284,7 @@ The `tuningConfig` is optional. If no `tuningConfig` is specified, default param |`indexSpecForIntermediatePersists`|Object|Defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. This can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [IndexSpec](#indexspec) for possible values.| no (default = same as `indexSpec`)| |`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| |`handoffConditionTimeout`|Long| Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.| no (default == 0)| -|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kinesis messages that are no longer available.

If false, the exception bubbles up, causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially using the [Reset Supervisor API](../../api-reference/api-reference.md#supervisors). This mode is useful for production, since it highlights issues with ingestion.

If true, Druid automatically resets to the earliest or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if true, latest if false). Note that this can lead to data being *DROPPED* (if `useEarliestSequenceNumber` is false) or *DUPLICATED* (if `useEarliestSequenceNumber` is true) without your knowledge. Druid will log messages indicating that a reset has occurred without interrupting ingestion. This mode is useful for non-production situations since it enables Druid to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.|no (default == false)| +|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kinesis messages that are no longer available.

If false, the exception bubbles up, causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it highlights issues with ingestion.

If true, Druid automatically resets to the earliest or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if true, latest if false). Note that this can lead to data being *DROPPED* (if `useEarliestSequenceNumber` is false) or *DUPLICATED* (if `useEarliestSequenceNumber` is true) without your knowledge. Druid will log messages indicating that a reset has occurred without interrupting ingestion. This mode is useful for non-production situations since it enables Druid to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.|no (default == false)| |`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If set to false, the indexing task will attempt to reset the current sequence number (or not), depending on the value of `resetOffsetAutomatically`.|no (default == false)| |`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|no (default == min(10, taskCount))| |`chatAsync`|Boolean| If true, the supervisor uses asynchronous communication with indexing tasks and ignores the `chatThreads` parameter. If false, the supervisor uses synchronous communication in a thread pool of size `chatThreads`.| no (default == true)| @@ -338,7 +338,7 @@ For Concise bitmaps: ## Operations This section describes how some supervisor APIs work in Kinesis Indexing Service. -For all supervisor APIs, check [Supervisor APIs](../../api-reference/api-reference.md#supervisors). +For all supervisor APIs, check [Supervisor API reference](../../api-reference/supervisor-api.md). ### AWS Authentication diff --git a/docs/ingestion/faq.md b/docs/ingestion/faq.md index b679c2eb155..24e119585ab 100644 --- a/docs/ingestion/faq.md +++ b/docs/ingestion/faq.md @@ -57,15 +57,15 @@ Make sure to include the `druid-hdfs-storage` and all the hadoop configuration, You can verify if segments created by a recent ingestion task are loaded onto historicals and available for querying using the following workflow. 1. Submit your ingestion task. -2. Repeatedly poll the [Overlord's tasks API](../api-reference/api-reference.md#tasks) ( `/druid/indexer/v1/task/{taskId}/status`) until your task is shown to be successfully completed. -3. Poll the [Segment Loading by Datasource API](../api-reference/api-reference.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with +2. Repeatedly poll the [Overlord's tasks API](../api-reference/tasks-api.md) ( `/druid/indexer/v1/task/{taskId}/status`) until your task is shown to be successfully completed. +3. Poll the [Segment Loading by Datasource API](../api-reference/legacy-metadata-api.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with `forceMetadataRefresh=true` and `interval=` once. (Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms of the load on the metadata store but is necessary to make sure that we verify all the latest segments' load status) If there are segments not yet loaded, continue to step 4, otherwise you can now query the data. -4. Repeatedly poll the [Segment Loading by Datasource API](../api-reference/api-reference.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with +4. Repeatedly poll the [Segment Loading by Datasource API](../api-reference/legacy-metadata-api.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with `forceMetadataRefresh=false` and `interval=`. Continue polling until all segments are loaded. Once all segments are loaded you can now query the data. -Note that this workflow only guarantees that the segments are available at the time of the [Segment Loading by Datasource API](../api-reference/api-reference.md#segment-loading-by-datasource) call. Segments can still become missing because of historical process failures or any other reasons afterward. +Note that this workflow only guarantees that the segments are available at the time of the [Segment Loading by Datasource API](../api-reference/legacy-metadata-api.md#segment-loading-by-datasource) call. Segments can still become missing because of historical process failures or any other reasons afterward. ## I don't see my Druid segments on my Historical processes diff --git a/docs/ingestion/hadoop.md b/docs/ingestion/hadoop.md index c75bccc8578..cb7f0308331 100644 --- a/docs/ingestion/hadoop.md +++ b/docs/ingestion/hadoop.md @@ -28,7 +28,7 @@ instance of a Druid [Overlord](../design/overlord.md). Please refer to our [Hado comparisons between Hadoop-based, native batch (simple), and native batch (parallel) ingestion. To run a Hadoop-based ingestion task, write an ingestion spec as specified below. Then POST it to the -[`/druid/indexer/v1/task`](../api-reference/api-reference.md#tasks) endpoint on the Overlord, or use the +[`/druid/indexer/v1/task`](../api-reference/tasks-api.md) endpoint on the Overlord, or use the `bin/post-index-task` script included with Druid. ## Tutorial diff --git a/docs/ingestion/index.md b/docs/ingestion/index.md index e483f4ad399..007c2e93cd9 100644 --- a/docs/ingestion/index.md +++ b/docs/ingestion/index.md @@ -69,7 +69,7 @@ runs for the duration of the job. | **Method** | [Native batch](./native-batch.md) | [SQL](../multi-stage-query/index.md) | [Hadoop-based](hadoop.md) | |---|-----|--------------|------------| | **Controller task type** | `index_parallel` | `query_controller` | `index_hadoop` | -| **How you submit it** | Send an `index_parallel` spec to the [task API](../api-reference/api-reference.md#tasks). | Send an [INSERT](../multi-stage-query/concepts.md#insert) or [REPLACE](../multi-stage-query/concepts.md#replace) statement to the [SQL task API](../api-reference/sql-ingestion-api.md#submit-a-query). | Send an `index_hadoop` spec to the [task API](../api-reference/api-reference.md#tasks). | +| **How you submit it** | Send an `index_parallel` spec to the [Tasks API](../api-reference/tasks-api.md). | Send an [INSERT](../multi-stage-query/concepts.md#insert) or [REPLACE](../multi-stage-query/concepts.md#replace) statement to the [SQL task API](../api-reference/sql-ingestion-api.md#submit-a-query). | Send an `index_hadoop` spec to the [Tasks API](../api-reference/tasks-api.md). | | **Parallelism** | Using subtasks, if [`maxNumConcurrentSubTasks`](native-batch.md#tuningconfig) is greater than 1. | Using `query_worker` subtasks. | Using YARN. | | **Fault tolerance** | Workers automatically relaunched upon failure. Controller task failure leads to job failure. | Controller or worker task failure leads to job failure. | YARN containers automatically relaunched upon failure. Controller task failure leads to job failure. | | **Can append?** | Yes. | Yes (INSERT). | No. | diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md index 7d6b9f78842..6741921f4fa 100644 --- a/docs/ingestion/native-batch.md +++ b/docs/ingestion/native-batch.md @@ -41,7 +41,7 @@ For related information on batch indexing, see: To run either kind of native batch indexing task you can: - Use the **Load Data** UI in the web console to define and submit an ingestion spec. -- Define an ingestion spec in JSON based upon the [examples](#parallel-indexing-example) and reference topics for batch indexing. Then POST the ingestion spec to the [Indexer API endpoint](../api-reference/api-reference.md#tasks), +- Define an ingestion spec in JSON based upon the [examples](#parallel-indexing-example) and reference topics for batch indexing. Then POST the ingestion spec to the [Tasks API endpoint](../api-reference/tasks-api.md), `/druid/indexer/v1/task`, the Overlord service. Alternatively you can use the indexing script included with Druid at `bin/post-index-task`. ## Parallel task indexing diff --git a/docs/ingestion/tasks.md b/docs/ingestion/tasks.md index 597aea05bb0..eb32d04e7c2 100644 --- a/docs/ingestion/tasks.md +++ b/docs/ingestion/tasks.md @@ -26,7 +26,7 @@ sidebar_label: Task reference Tasks do all [ingestion](index.md)-related work in Druid. For batch ingestion, you will generally submit tasks directly to Druid using the -[Task APIs](../api-reference/api-reference.md#tasks). For streaming ingestion, tasks are generally submitted for you by a +[Tasks APIs](../api-reference/tasks-api.md). For streaming ingestion, tasks are generally submitted for you by a supervisor. ## Task API @@ -34,7 +34,7 @@ supervisor. Task APIs are available in two main places: - The [Overlord](../design/overlord.md) process offers HTTP APIs to submit tasks, cancel tasks, check their status, -review logs and reports, and more. Refer to the [Tasks API reference page](../api-reference/api-reference.md#tasks) for a +review logs and reports, and more. Refer to the [Tasks API reference](../api-reference/tasks-api.md) for a full list. - Druid SQL includes a [`sys.tasks`](../querying/sql-metadata-tables.md#tasks-table) table that provides information about currently running tasks. This table is read-only, and has a limited (but useful!) subset of the full information available through @@ -406,7 +406,7 @@ The task then starts creating logs in a local directory of the middle manager (o When the task completes - whether it succeeds or fails - the middle manager (or indexer) will push the task log file into the location specified in [`druid.indexer.logs`](../configuration/index.md#task-logging). -Task logs on the Druid web console are retrieved via an [API](../api-reference/api-reference.md#overlord) on the Overlord. It automatically detects where the log file is, either in the middleManager / indexer or in long-term storage, and passes it back. +Task logs on the Druid web console are retrieved via an [API](../api-reference/service-status-api.md#overlord) on the Overlord. It automatically detects where the log file is, either in the middleManager / indexer or in long-term storage, and passes it back. If you don't see the log file in long-term storage, it means either: diff --git a/docs/operations/rule-configuration.md b/docs/operations/rule-configuration.md index 0bf803355aa..8d12beac96c 100644 --- a/docs/operations/rule-configuration.md +++ b/docs/operations/rule-configuration.md @@ -38,7 +38,7 @@ Retention rules are persistent: they remain in effect until you change them. Dru ## Set retention rules -You can use the Druid [web console](./web-console.md) or the [Coordinator API](../api-reference/api-reference.md#coordinator) to create and manage retention rules. +You can use the Druid [web console](./web-console.md) or the [Service status API reference](../api-reference/service-status-api.md#coordinator) to create and manage retention rules. ### Use the web console diff --git a/docs/querying/lookups.md b/docs/querying/lookups.md index 7ed8fed09d5..d69f31619a5 100644 --- a/docs/querying/lookups.md +++ b/docs/querying/lookups.md @@ -146,257 +146,8 @@ The Coordinator periodically checks if any of the processes need to load/drop lo Please note that only 2 simultaneous lookup configuration propagation requests can be concurrently handled by a single query serving process. This limit is applied to prevent lookup handling from consuming too many server HTTP connections. -## API for configuring lookups - -### Bulk update -Lookups can be updated in bulk by posting a JSON object to `/druid/coordinator/v1/lookups/config`. The format of the json object is as follows: - -```json -{ - "": { - "": { - "version": "", - "lookupExtractorFactory": { - "type": "", - "": "" - } - } - } -} -``` - -Note that "version" is an arbitrary string assigned by the user, when making updates to existing lookup then user would need to specify a lexicographically higher version. - -For example, a config might look something like: - -```json -{ - "__default": { - "country_code": { - "version": "v0", - "lookupExtractorFactory": { - "type": "map", - "map": { - "77483": "United States" - } - } - }, - "site_id": { - "version": "v0", - "lookupExtractorFactory": { - "type": "cachedNamespace", - "extractionNamespace": { - "type": "jdbc", - "connectorConfig": { - "createTables": true, - "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", - "user": "druid", - "password": "diurd" - }, - "table": "lookupTable", - "keyColumn": "country_id", - "valueColumn": "country_name", - "tsColumn": "timeColumn" - }, - "firstCacheTimeout": 120000, - "injective": true - } - }, - "site_id_customer1": { - "version": "v0", - "lookupExtractorFactory": { - "type": "map", - "map": { - "847632": "Internal Use Only" - } - } - }, - "site_id_customer2": { - "version": "v0", - "lookupExtractorFactory": { - "type": "map", - "map": { - "AHF77": "Home" - } - } - } - }, - "realtime_customer1": { - "country_code": { - "version": "v0", - "lookupExtractorFactory": { - "type": "map", - "map": { - "77483": "United States" - } - } - }, - "site_id_customer1": { - "version": "v0", - "lookupExtractorFactory": { - "type": "map", - "map": { - "847632": "Internal Use Only" - } - } - } - }, - "realtime_customer2": { - "country_code": { - "version": "v0", - "lookupExtractorFactory": { - "type": "map", - "map": { - "77483": "United States" - } - } - }, - "site_id_customer2": { - "version": "v0", - "lookupExtractorFactory": { - "type": "map", - "map": { - "AHF77": "Home" - } - } - } - } -} -``` - -All entries in the map will UPDATE existing entries. No entries will be deleted. - -### Update lookup - -A `POST` to a particular lookup extractor factory via `/druid/coordinator/v1/lookups/config/{tier}/{id}` creates or updates that specific extractor factory. - -For example, a post to `/druid/coordinator/v1/lookups/config/realtime_customer1/site_id_customer1` might contain the following: - -```json -{ - "version": "v1", - "lookupExtractorFactory": { - "type": "map", - "map": { - "847632": "Internal Use Only" - } - } -} -``` - -This will replace the `site_id_customer1` lookup in the `realtime_customer1` with the definition above. - -Assign a unique version identifier each time you update a lookup extractor factory. Otherwise the call will fail. - -### Get all lookups - -A `GET` to `/druid/coordinator/v1/lookups/config/all` will return all known lookup specs for all tiers. - -### Get lookup - -A `GET` to a particular lookup extractor factory is accomplished via `/druid/coordinator/v1/lookups/config/{tier}/{id}` - -Using the prior example, a `GET` to `/druid/coordinator/v1/lookups/config/realtime_customer2/site_id_customer2` should return - -```json -{ - "version": "v1", - "lookupExtractorFactory": { - "type": "map", - "map": { - "AHF77": "Home" - } - } -} -``` - -### Delete lookup - -A `DELETE` to `/druid/coordinator/v1/lookups/config/{tier}/{id}` will remove that lookup from the cluster. If it was last lookup in the tier, then tier is deleted as well. - -### Delete tier - -A `DELETE` to `/druid/coordinator/v1/lookups/config/{tier}` will remove that tier from the cluster. - -### List tier names - -A `GET` to `/druid/coordinator/v1/lookups/config` will return a list of known tier names in the dynamic configuration. -To discover a list of tiers currently active in the cluster in addition to ones known in the dynamic configuration, the parameter `discover=true` can be added as per `/druid/coordinator/v1/lookups/config?discover=true`. - -### List lookup names - -A `GET` to `/druid/coordinator/v1/lookups/config/{tier}` will return a list of known lookup names for that tier. - -These end points can be used to get the propagation status of configured lookups to processes using lookups such as Historicals. - -## API for lookup status - -### List load status of all lookups - -`GET /druid/coordinator/v1/lookups/status` with optional query parameter `detailed`. - -### List load status of lookups in a tier - -`GET /druid/coordinator/v1/lookups/status/{tier}` with optional query parameter `detailed`. - -### List load status of single lookup - -`GET /druid/coordinator/v1/lookups/status/{tier}/{lookup}` with optional query parameter `detailed`. - -### List lookup state of all processes - -`GET /druid/coordinator/v1/lookups/nodeStatus` with optional query parameter `discover` to discover tiers advertised by other Druid nodes, or by default, returning all configured lookup tiers. The default response will also include the lookups which are loaded, being loaded, or being dropped on each node, for each tier, including the complete lookup spec. Add the optional query parameter `detailed=false` to only include the 'version' of the lookup instead of the complete spec. - -### List lookup state of processes in a tier - -`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}` - -### List lookup state of single process - -`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}/{host:port}` - -## Internal API - -The Peon, Router, Broker, and Historical processes all have the ability to consume lookup configuration. -There is an internal API these processes use to list/load/drop their lookups starting at `/druid/listen/v1/lookups`. -These follow the same convention for return values as the cluster wide dynamic configuration. Following endpoints -can be used for debugging purposes but not otherwise. - -### Get lookups - -A `GET` to the process at `/druid/listen/v1/lookups` will return a json map of all the lookups currently active on the process. -The return value will be a json map of the lookups to their extractor factories. - -```json -{ - "site_id_customer2": { - "version": "v1", - "lookupExtractorFactory": { - "type": "map", - "map": { - "AHF77": "Home" - } - } - } -} -``` - -### Get lookup - -A `GET` to the process at `/druid/listen/v1/lookups/some_lookup_name` will return the LookupExtractorFactory for the lookup identified by `some_lookup_name`. -The return value will be the json representation of the factory. - -```json -{ - "version": "v1", - "lookupExtractorFactory": { - "type": "map", - "map": { - "AHF77": "Home" - } - } -} -``` +## API +See [Lookups API](../api-reference/lookups-api.md) for reference on configuring lookups and lookup status. ## Configuration diff --git a/website/sidebars.json b/website/sidebars.json index 8f4077e808f..458d2bfe033 100644 --- a/website/sidebars.json +++ b/website/sidebars.json @@ -164,10 +164,33 @@ } ], "API reference":[ - "api-reference/sql-api", - "api-reference/sql-ingestion-api", - "api-reference/sql-jdbc", - "api-reference/api-reference" + "api-reference/api-reference", + { + "type": "subcategory", + "label": "HTTP APIs", + "ids": [ + "api-reference/sql-api", + "api-reference/sql-ingestion-api", + "api-reference/json-querying-api", + "api-reference/tasks-api", + "api-reference/supervisor-api", + "api-reference/retention-rules-api", + "api-reference/data-management-api", + "api-reference/automatic-compaction-api", + "api-reference/lookups-api", + "api-reference/service-status-api", + "api-reference/dynamic-configuration-api", + "api-reference/legacy-metadata-api" + ] + }, + { + "type": "subcategory", + "label": "Java APIs", + "ids": [ + "api-reference/sql-jdbc" + ] + } + ], "Configuration": [ "configuration/index", From 79bff4bbf7b317ec0d552d595e07b6da29b810b4 Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Mon, 26 Jun 2023 20:01:11 -0700 Subject: [PATCH 32/74] Improvements to `EXPLAIN PLAN` attributes (#14441) * Updates: use the target table directly, sanitized replace time chunks and clustered by cols. * Add DruidSqlParserUtil and tests. * minor refactor * Use SqlUtil.isLiteral * Throw ValidationException if CLUSTERED BY column descending order is specified. - Fails query planning * Some more tests. * fixup existing comment * Update comment * checkstyle fix: remove unused imports * Remove InsertCannotOrderByDescendingFault and deprecate the fault in readme. * minor naming * move deprecated field to the bottom * update docs. * add one more example. * Collapsible query and result * checkstyle fixes * Code cleanup * order by changes * conditionally set attributes only for explain queries. * Cleaner ordinal check. * Add limit test and update javadoc. * Commentary and minor adjustments. * Checkstyle fixes. * One more checkArg. * add unexpected kind to exception. --- docs/querying/sql-translation.md | 260 +++++++++++++++++- .../sql/calcite/parser/DruidSqlIngest.java | 3 +- .../calcite/parser/DruidSqlParserUtils.java | 112 +++++++- .../sql/calcite/planner/DruidPlanner.java | 4 +- .../calcite/planner/ExplainAttributes.java | 25 +- .../sql/calcite/planner/IngestHandler.java | 23 +- .../sql/calcite/CalciteInsertDmlTest.java | 4 +- .../sql/calcite/CalciteReplaceDmlTest.java | 110 +++++++- .../parser/DruidSqlParserUtilsTest.java | 213 ++++++++++++++ .../calcite/parser/DruidSqlUnparseTest.java | 3 +- .../planner/ExplainAttributesTest.java | 87 ++++-- 11 files changed, 771 insertions(+), 73 deletions(-) diff --git a/docs/querying/sql-translation.md b/docs/querying/sql-translation.md index 5126b9fc350..5528db95323 100644 --- a/docs/querying/sql-translation.md +++ b/docs/querying/sql-translation.md @@ -76,6 +76,8 @@ EXPLAIN PLAN statements return: Example 1: EXPLAIN PLAN for a `SELECT` query on the `wikipedia` datasource: +
Show the query + ```sql EXPLAIN PLAN FOR SELECT @@ -85,9 +87,12 @@ FROM wikipedia WHERE channel IN (SELECT page FROM wikipedia GROUP BY page ORDER BY COUNT(*) DESC LIMIT 10) GROUP BY channel ``` +
The above EXPLAIN PLAN query returns the following result: +
Show the result + ```json [ [ @@ -224,13 +229,15 @@ The above EXPLAIN PLAN query returns the following result: } ] ``` +
-Example 2: EXPLAIN PLAN for a `REPLACE` query that replaces all the data in the `wikipedia` datasource: +Example 2: EXPLAIN PLAN for an `INSERT` query that inserts data into the `wikipedia` datasource: + +
Show the query ```sql EXPLAIN PLAN FOR -REPLACE INTO wikipedia -OVERWRITE ALL +INSERT INTO wikipedia2 SELECT TIME_PARSE("timestamp") AS __time, namespace, @@ -247,11 +254,14 @@ FROM TABLE( '[{"name":"timestamp","type":"string"},{"name":"namespace","type":"string"},{"name":"cityName","type":"string"},{"name":"countryName","type":"string"},{"name":"regionIsoCode","type":"string"},{"name":"metroCode","type":"long"},{"name":"countryIsoCode","type":"string"},{"name":"regionName","type":"string"}]' ) ) -PARTITIONED BY HOUR -CLUSTERED BY cityName +PARTITIONED BY ALL ``` +
-The above EXPLAIN PLAN query returns the following result: + +The above EXPLAIN PLAN returns the following result: + +
Show the result ```json [ @@ -323,12 +333,229 @@ The above EXPLAIN PLAN query returns the following result: } ], "resultFormat": "compactedList", - "orderBy": [ + "columns": [ + "cityName", + "countryIsoCode", + "countryName", + "metroCode", + "namespace", + "regionIsoCode", + "regionName", + "v0" + ], + "legacy": false, + "context": { + "finalizeAggregations": false, + "forceExpressionVirtualColumns": true, + "groupByEnableMultiValueUnnesting": false, + "maxNumTasks": 5, + "multiStageQuery": true, + "queryId": "42e3de2b-daaf-40f9-a0e7-2c6184529ea3", + "scanSignature": "[{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"v0\",\"type\":\"LONG\"}]", + "sqlInsertSegmentGranularity": "{\"type\":\"all\"}", + "sqlQueryId": "42e3de2b-daaf-40f9-a0e7-2c6184529ea3", + "useNativeQueryExplain": true + }, + "granularity": { + "type": "all" + } + }, + "signature": [ + { + "name": "v0", + "type": "LONG" + }, + { + "name": "namespace", + "type": "STRING" + }, + { + "name": "cityName", + "type": "STRING" + }, + { + "name": "countryName", + "type": "STRING" + }, + { + "name": "regionIsoCode", + "type": "STRING" + }, + { + "name": "metroCode", + "type": "LONG" + }, + { + "name": "countryIsoCode", + "type": "STRING" + }, + { + "name": "regionName", + "type": "STRING" + } + ], + "columnMappings": [ + { + "queryColumn": "v0", + "outputColumn": "__time" + }, + { + "queryColumn": "namespace", + "outputColumn": "namespace" + }, + { + "queryColumn": "cityName", + "outputColumn": "cityName" + }, + { + "queryColumn": "countryName", + "outputColumn": "countryName" + }, + { + "queryColumn": "regionIsoCode", + "outputColumn": "regionIsoCode" + }, + { + "queryColumn": "metroCode", + "outputColumn": "metroCode" + }, + { + "queryColumn": "countryIsoCode", + "outputColumn": "countryIsoCode" + }, + { + "queryColumn": "regionName", + "outputColumn": "regionName" + } + ] + } + ], + [ + { + "name": "EXTERNAL", + "type": "EXTERNAL" + }, + { + "name": "wikipedia", + "type": "DATASOURCE" + } + ], + { + "statementType": "INSERT", + "targetDataSource": "wikipedia", + "partitionedBy": { + "type": "all" + } + } +] +``` +
+ +Example 3: EXPLAIN PLAN for a `REPLACE` query that replaces all the data in the `wikipedia` datasource with a `DAY` +time partitioning, and `cityName` and `countryName` as the clustering columns: + +
Show the query + +```sql +EXPLAIN PLAN FOR +REPLACE INTO wikipedia +OVERWRITE ALL +SELECT + TIME_PARSE("timestamp") AS __time, + namespace, + cityName, + countryName, + regionIsoCode, + metroCode, + countryIsoCode, + regionName +FROM TABLE( + EXTERN( + '{"type":"http","uris":["https://druid.apache.org/data/wikipedia.json.gz"]}', + '{"type":"json"}', + '[{"name":"timestamp","type":"string"},{"name":"namespace","type":"string"},{"name":"cityName","type":"string"},{"name":"countryName","type":"string"},{"name":"regionIsoCode","type":"string"},{"name":"metroCode","type":"long"},{"name":"countryIsoCode","type":"string"},{"name":"regionName","type":"string"}]' + ) + ) +PARTITIONED BY DAY +CLUSTERED BY cityName, countryName +``` +
+ + +The above EXPLAIN PLAN query returns the following result: + +
Show the result + +```json +[ + [ + { + "query": { + "queryType": "scan", + "dataSource": { + "type": "external", + "inputSource": { + "type": "http", + "uris": [ + "https://druid.apache.org/data/wikipedia.json.gz" + ] + }, + "inputFormat": { + "type": "json", + "keepNullColumns": false, + "assumeNewlineDelimited": false, + "useJsonNodeReader": false + }, + "signature": [ + { + "name": "timestamp", + "type": "STRING" + }, + { + "name": "namespace", + "type": "STRING" + }, + { + "name": "cityName", + "type": "STRING" + }, + { + "name": "countryName", + "type": "STRING" + }, + { + "name": "regionIsoCode", + "type": "STRING" + }, + { + "name": "metroCode", + "type": "LONG" + }, + { + "name": "countryIsoCode", + "type": "STRING" + }, + { + "name": "regionName", + "type": "STRING" + } + ] + }, + "intervals": { + "type": "intervals", + "intervals": [ + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" + ] + }, + "virtualColumns": [ { - "columnName": "cityName", - "order": "ascending" + "type": "expression", + "name": "v0", + "expression": "timestamp_parse(\"timestamp\",null,'UTC')", + "outputType": "LONG" } ], + "resultFormat": "compactedList", "columns": [ "cityName", "countryIsoCode", @@ -344,10 +571,10 @@ The above EXPLAIN PLAN query returns the following result: "finalizeAggregations": false, "groupByEnableMultiValueUnnesting": false, "maxNumTasks": 5, - "queryId": "b474c0d5-a5ce-432d-be94-535ccdb7addc", + "queryId": "d88e0823-76d4-40d9-a1a7-695c8577b79f", "scanSignature": "[{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"v0\",\"type\":\"LONG\"}]", - "sqlInsertSegmentGranularity": "\"HOUR\"", - "sqlQueryId": "b474c0d5-a5ce-432d-be94-535ccdb7addc", + "sqlInsertSegmentGranularity": "\"DAY\"", + "sqlQueryId": "d88e0823-76d4-40d9-a1a7-695c8577b79f", "sqlReplaceTimeChunks": "all" }, "granularity": { @@ -437,13 +664,16 @@ The above EXPLAIN PLAN query returns the following result: { "statementType": "REPLACE", "targetDataSource": "wikipedia", - "partitionedBy": "HOUR", - "clusteredBy": "`cityName`", - "replaceTimeChunks": "'ALL'" + "partitionedBy": "DAY", + "clusteredBy": ["cityName","countryName"], + "replaceTimeChunks": "all" } ] ``` +
+ + In this case the JOIN operator gets translated to a `join` datasource. See the [Join translation](#joins) section for more details about how this works. diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlIngest.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlIngest.java index 26f019e0a15..146d13673bd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlIngest.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlIngest.java @@ -42,7 +42,8 @@ public abstract class DruidSqlIngest extends SqlInsert @Nullable protected final SqlNodeList clusteredBy; - public DruidSqlIngest(SqlParserPos pos, + public DruidSqlIngest( + SqlParserPos pos, SqlNodeList keywords, SqlNode targetTable, SqlNode source, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java index 5f11c6f836a..36fe62e2db3 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java @@ -22,6 +22,7 @@ package org.apache.druid.sql.calcite.parser; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlIdentifier; @@ -30,8 +31,13 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlNumericLiteral; +import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlOrderBy; +import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.SqlTimestampLiteral; +import org.apache.calcite.sql.SqlUtil; +import org.apache.calcite.sql.dialect.CalciteSqlDialect; import org.apache.calcite.tools.ValidationException; import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidSqlInput; @@ -57,6 +63,7 @@ import org.joda.time.Interval; import org.joda.time.Period; import org.joda.time.base.AbstractInterval; +import javax.annotation.Nullable; import java.sql.Timestamp; import java.time.ZonedDateTime; import java.util.ArrayList; @@ -200,7 +207,7 @@ public class DruidSqlParserUtils } /** - * This method validates and converts a {@link SqlNode} representing a query into an optmizied list of intervals to + * This method validates and converts a {@link SqlNode} representing a query into an optimized list of intervals to * be used in creating an ingestion spec. If the sqlNode is an SqlLiteral of {@link #ALL}, returns a singleton list of * "ALL". Otherwise, it converts and optimizes the query using {@link MoveTimeFiltersToIntervals} into a list of * intervals which contain all valid values of time as per the query. @@ -302,6 +309,109 @@ public class DruidSqlParserUtils ); } + /** + * Return resolved clustered by column output names. + * For example, consider the following SQL: + *
+   * EXPLAIN PLAN FOR
+   * INSERT INTO w000
+   * SELECT
+   *  TIME_PARSE("timestamp") AS __time,
+   *  page AS page_alias,
+   *  FLOOR("cost"),
+   *  country,
+   *  citName
+   * FROM ...
+   * PARTITIONED BY DAY
+   * CLUSTERED BY 1, 2, 3, cityName
+   * 
+ * + *

+ * The function will return the following clusteredBy columns for the above SQL: ["__time", "page_alias", "FLOOR(\"cost\")", cityName"] + * Any ordinal and expression specified in the CLUSTERED BY clause will resolve to the final output column name. + *

+ * @param clusteredByNodes List of {@link SqlNode}s representing columns to be clustered by. + * @param sourceNode The select or order by source node. + */ + @Nullable + public static List resolveClusteredByColumnsToOutputColumns(SqlNodeList clusteredByNodes, SqlNode sourceNode) + { + // CLUSTERED BY is an optional clause + if (clusteredByNodes == null) { + return null; + } + + Preconditions.checkArgument( + sourceNode instanceof SqlSelect || sourceNode instanceof SqlOrderBy, + "Source node must be either SqlSelect or SqlOrderBy, but found [%s]", + sourceNode == null ? null : sourceNode.getKind() + ); + + final SqlSelect selectNode = (sourceNode instanceof SqlSelect) ? (SqlSelect) sourceNode + : (SqlSelect) ((SqlOrderBy) sourceNode).query; + final List selectList = selectNode.getSelectList().getList(); + final List retClusteredByNames = new ArrayList<>(); + + for (SqlNode clusteredByNode : clusteredByNodes) { + + if (SqlUtil.isLiteral(clusteredByNode)) { + // The node is a literal number -- an ordinal is specified in the CLUSTERED BY clause. Validate and lookup the + // ordinal in the select list. + int ordinal = ((SqlNumericLiteral) clusteredByNode).getValueAs(Integer.class); + if (ordinal < 1 || ordinal > selectList.size()) { + throw InvalidSqlInput.exception( + "Ordinal[%d] specified in the CLUSTERED BY clause is invalid. It must be between 1 and %d.", + ordinal, + selectList.size() + ); + } + SqlNode node = selectList.get(ordinal - 1); + + if (node instanceof SqlBasicCall) { + retClusteredByNames.add(getColumnNameFromSqlCall(node)); + } else { + Preconditions.checkArgument( + node instanceof SqlIdentifier, + "Node must be a SqlIdentifier, but found [%s]", + node.getKind() + ); + SqlIdentifier n = ((SqlIdentifier) node); + retClusteredByNames.add(n.isSimple() ? n.getSimple() : n.names.get(1)); + } + } else if (clusteredByNode instanceof SqlBasicCall) { + // The node is an expression/operator. + retClusteredByNames.add(getColumnNameFromSqlCall(clusteredByNode)); + } else { + // The node is a simple SqlIdentifier, add the name. + Preconditions.checkArgument( + clusteredByNode instanceof SqlIdentifier, + "ClusteredBy node must be a SqlIdentifier, but found [%s]", + clusteredByNode.getKind() + ); + retClusteredByNames.add(clusteredByNode.toString()); + } + } + + return retClusteredByNames; + } + + private static String getColumnNameFromSqlCall(final SqlNode sqlCallNode) + { + Preconditions.checkArgument(sqlCallNode instanceof SqlBasicCall, "Node must be a SqlBasicCall type"); + + // The node may be an alias or expression, in which case we'll get the output name + SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlCallNode; + SqlOperator operator = (sqlBasicCall).getOperator(); + if (operator instanceof SqlAsOperator) { + // Get the output name for the alias operator. + SqlNode sqlNode = (sqlBasicCall).getOperandList().get(1); + return sqlNode.toString(); + } else { + // Return the expression as-is. + return sqlCallNode.toSqlString(CalciteSqlDialect.DEFAULT).toString(); + } + } + /** * Validates the clustered by columns to ensure that it does not contain DESCENDING order columns. * diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java index fc6c08a0403..7d7bdc8d545 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java @@ -151,7 +151,9 @@ public class DruidPlanner implements Closeable handler = createHandler(root); handler.validate(); plannerContext.setResourceActions(handler.resourceActions()); - plannerContext.setExplainAttributes(handler.explainAttributes()); + if (root.getKind() == SqlKind.EXPLAIN) { + plannerContext.setExplainAttributes(handler.explainAttributes()); + } state = State.VALIDATED; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExplainAttributes.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExplainAttributes.java index 8d040f23fa5..e2ae4fa7a10 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExplainAttributes.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExplainAttributes.java @@ -21,11 +21,10 @@ package org.apache.druid.sql.calcite.planner; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNodeList; import org.apache.druid.java.util.common.granularity.Granularity; import javax.annotation.Nullable; +import java.util.List; /** * ExplainAttributes holds the attributes of a SQL statement that is used in the EXPLAIN PLAN result. @@ -35,23 +34,23 @@ public final class ExplainAttributes private final String statementType; @Nullable - private final SqlNode targetDataSource; + private final String targetDataSource; @Nullable private final Granularity partitionedBy; @Nullable - private final SqlNodeList clusteredBy; + private final List clusteredBy; @Nullable - private final SqlNode replaceTimeChunks; + private final String replaceTimeChunks; public ExplainAttributes( @JsonProperty("statementType") final String statementType, - @JsonProperty("targetDataSource") @Nullable final SqlNode targetDataSource, + @JsonProperty("targetDataSource") @Nullable final String targetDataSource, @JsonProperty("partitionedBy") @Nullable final Granularity partitionedBy, - @JsonProperty("clusteredBy") @Nullable final SqlNodeList clusteredBy, - @JsonProperty("replaceTimeChunks") @Nullable final SqlNode replaceTimeChunks + @JsonProperty("clusteredBy") @Nullable final List clusteredBy, + @JsonProperty("replaceTimeChunks") @Nullable final String replaceTimeChunks ) { this.statementType = statementType; @@ -62,7 +61,7 @@ public final class ExplainAttributes } /** - * @return the statement kind of a SQL statement. For example, SELECT, INSERT, or REPLACE. + * @return the SQL statement type. For example, SELECT, INSERT, or REPLACE. */ @JsonProperty public String getStatementType() @@ -79,7 +78,7 @@ public final class ExplainAttributes @JsonInclude(JsonInclude.Include.NON_NULL) public String getTargetDataSource() { - return targetDataSource == null ? null : targetDataSource.toString(); + return targetDataSource; } /** @@ -101,9 +100,9 @@ public final class ExplainAttributes @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public String getClusteredBy() + public List getClusteredBy() { - return clusteredBy == null ? null : clusteredBy.toString(); + return clusteredBy; } /** @@ -115,7 +114,7 @@ public final class ExplainAttributes @JsonInclude(JsonInclude.Include.NON_NULL) public String getReplaceTimeChunks() { - return replaceTimeChunks == null ? null : replaceTimeChunks.toString(); + return replaceTimeChunks; } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java index 52b4efcfeb0..2ce19acbbac 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java @@ -275,9 +275,9 @@ public abstract class IngestHandler extends QueryHandler { return new ExplainAttributes( DruidSqlInsert.OPERATOR.getName(), - sqlNode.getTargetTable(), - sqlNode.getPartitionedBy(), - sqlNode.getClusteredBy(), + targetDatasource, + ingestionGranularity, + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(sqlNode.getClusteredBy(), sqlNode.getSource()), null ); } @@ -289,7 +289,7 @@ public abstract class IngestHandler extends QueryHandler protected static class ReplaceHandler extends IngestHandler { private final DruidSqlReplace sqlNode; - private List replaceIntervals; + private String replaceIntervals; public ReplaceHandler( SqlStatementHandler.HandlerContext handlerContext, @@ -329,16 +329,17 @@ public abstract class IngestHandler extends QueryHandler ); } - replaceIntervals = DruidSqlParserUtils.validateQueryAndConvertToIntervals( + List replaceIntervalsList = DruidSqlParserUtils.validateQueryAndConvertToIntervals( replaceTimeQuery, ingestionGranularity, handlerContext.timeZone() ); super.validate(); - if (replaceIntervals != null) { + if (replaceIntervalsList != null) { + replaceIntervals = String.join(",", replaceIntervalsList); handlerContext.queryContextMap().put( DruidSqlReplace.SQL_REPLACE_TIME_CHUNKS, - String.join(",", replaceIntervals) + replaceIntervals ); } } @@ -348,10 +349,10 @@ public abstract class IngestHandler extends QueryHandler { return new ExplainAttributes( DruidSqlReplace.OPERATOR.getName(), - sqlNode.getTargetTable(), - sqlNode.getPartitionedBy(), - sqlNode.getClusteredBy(), - sqlNode.getReplaceTimeQuery() + targetDatasource, + ingestionGranularity, + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(sqlNode.getClusteredBy(), sqlNode.getSource()), + replaceIntervals ); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index 7fb52843b4e..1679b86fb06 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -657,7 +657,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest skipVectorize(); final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"druid.dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":\"2, `dim1`, CEIL(`m2`)\"}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"floor_m1\",\"dim1\",\"CEIL(\\\"m2\\\")\"]}"; final String sql = "EXPLAIN PLAN FOR INSERT INTO druid.dst " + "SELECT __time, FLOOR(m1) as floor_m1, dim1, CEIL(m2) as ceil_m2 FROM foo " @@ -1126,7 +1126,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest + "}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"druid.dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":\"2, `dim1`, CEIL(`m2`)\"}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"floor_m1\",\"dim1\",\"CEIL(\\\"m2\\\")\"]}"; // Use testQuery for EXPLAIN (not testIngestionQuery). testQuery( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index d7ba655c1ef..282ea7d8251 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -654,7 +654,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest + "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"},\"replaceTimeChunks\":\"'ALL'\"}"; + final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"},\"replaceTimeChunks\":\"all\"}"; // Use testQuery for EXPLAIN (not testIngestionQuery). testQuery( @@ -732,12 +732,116 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"dim1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":\"`dim1`\",\"replaceTimeChunks\":\"`__time` >= TIMESTAMP '2000-01-01 00:00:00' AND `__time` < TIMESTAMP '2000-01-02 00:00:00'\"}"; + final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"dim1\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}"; final String sql = "EXPLAIN PLAN FOR" + " REPLACE INTO dst" + " OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00' " - + "SELECT * FROM foo PARTITIONED BY DAY CLUSTERED BY dim1"; + + "SELECT * FROM foo PARTITIONED BY DAY CLUSTERED BY dim1 ASC"; + // Use testQuery for EXPLAIN (not testIngestionQuery). + testQuery( + PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, + ImmutableMap.of("sqlQueryId", "dummy"), + Collections.emptyList(), + sql, + CalciteTests.SUPER_USER_AUTH_RESULT, + ImmutableList.of(), + new DefaultResultsVerifier( + ImmutableList.of( + new Object[]{ + legacyExplanation, + resources, + attributes + } + ), + null + ), + null + ); + + testQuery( + PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, + ImmutableMap.of("sqlQueryId", "dummy"), + Collections.emptyList(), + sql, + CalciteTests.SUPER_USER_AUTH_RESULT, + ImmutableList.of(), + new DefaultResultsVerifier( + ImmutableList.of( + new Object[]{ + explanation, + resources, + attributes + } + ), + null + ), + null + ); + + // Not using testIngestionQuery, so must set didTest manually to satisfy the check in tearDown. + didTest = true; + } + + @Test + public void testExplainReplaceWithLimitAndClusteredByOrdinals() throws IOException + { + // Skip vectorization since otherwise the "context" will change for each subtest. + skipVectorize(); + + ObjectMapper queryJsonMapper = queryFramework().queryJsonMapper(); + final ScanQuery expectedQuery = newScanQueryBuilder() + .dataSource("foo") + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") + .limit(10) + .orderBy( + ImmutableList.of( + new ScanQuery.OrderBy("__time", ScanQuery.Order.ASCENDING), + new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING), + new ScanQuery.OrderBy("dim3", ScanQuery.Order.ASCENDING), + new ScanQuery.OrderBy("dim2", ScanQuery.Order.ASCENDING) + ) + ) + .context( + queryJsonMapper.readValue( + "{\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}", + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ) + ) + .build(); + + final String legacyExplanation = + "DruidQueryRel(query=[" + + queryJsonMapper.writeValueAsString(expectedQuery) + + "], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"; + + final String explanation = "[" + + "{\"query\":{\"queryType\":\"scan\",\"dataSource\":" + + "{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\"," + + "\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + + "\"resultFormat\":\"compactedList\",\"limit\":10," + + "\"orderBy\":[{\"columnName\":\"__time\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"}," + + "{\"columnName\":\"dim3\",\"order\":\"ascending\"},{\"columnName\":\"dim2\",\"order\":\"ascending\"}]," + + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," + + "\"legacy\":false,\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\"," + + "\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\"," + + "\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," + + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"}," + + "{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}]," + + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}," + + "{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"}," + + "{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; + final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; + final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"HOUR\"," + + "\"clusteredBy\":[\"__time\",\"dim1\",\"dim3\",\"dim2\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}"; + + final String sql = "EXPLAIN PLAN FOR" + + " REPLACE INTO dst" + + " OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00' " + + " SELECT * FROM foo LIMIT 10" + + " PARTITIONED BY HOUR CLUSTERED BY __time, dim1, 4, dim2"; + // Use testQuery for EXPLAIN (not testIngestionQuery). testQuery( PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java index 01f0544e156..b47e5bfeaa3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java @@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite.parser; import com.google.common.collect.ImmutableList; import org.apache.calcite.avatica.util.TimeUnit; +import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlIntervalQualifier; @@ -28,7 +29,9 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlPostfixOperator; +import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.druid.error.DruidException; @@ -43,6 +46,8 @@ import org.junit.experimental.runners.Enclosed; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.util.Arrays; + @RunWith(Enclosed.class) public class DruidSqlParserUtilsTest { @@ -125,6 +130,214 @@ public class DruidSqlParserUtilsTest } } + /** + * Test class that validates the resolution of "CLUSTERED BY" columns to output columns. + */ + public static class ResolveClusteredByColumnsTest + { + @Test + public void testNullClusteredByAndSource() + { + Assert.assertNull(DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(null, null)); + } + + @Test + public void testNullClusteredBy() + { + final SqlNodeList selectArgs = new SqlNodeList(SqlParserPos.ZERO); + selectArgs.add(new SqlIdentifier("__time", new SqlParserPos(0, 1))); + Assert.assertNull(DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns( + null, + new SqlSelect(SqlParserPos.ZERO, null, selectArgs, null, null, null, null, null, null, null, null) + ) + ); + } + + @Test + public void testNullSource() + { + final SqlNodeList args = new SqlNodeList(SqlParserPos.ZERO); + args.add(new SqlIdentifier("__time", SqlParserPos.ZERO)); + args.add(new SqlIntervalQualifier(TimeUnit.DAY, null, SqlParserPos.ZERO)); + + IllegalArgumentException iae = Assert.assertThrows( + IllegalArgumentException.class, + () -> DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(args, null) + ); + Assert.assertEquals("Source node must be either SqlSelect or SqlOrderBy, but found [null]", iae.getMessage()); + } + + @Test + public void testSimpleClusteredBy() + { + final SqlNodeList selectArgs = new SqlNodeList(SqlParserPos.ZERO); + selectArgs.add(new SqlIdentifier("__time", new SqlParserPos(0, 1))); + selectArgs.add(new SqlIdentifier("FOO", new SqlParserPos(0, 2))); + selectArgs.add(new SqlIdentifier("BOO", new SqlParserPos(0, 3))); + + final SqlSelect sqlSelect = new SqlSelect( + SqlParserPos.ZERO, + null, + selectArgs, + null, + null, + null, + null, + null, + null, + null, + null + ); + + final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); + clusteredByArgs.add(new SqlIdentifier("__time", SqlParserPos.ZERO)); + clusteredByArgs.add(new SqlIdentifier("FOO", SqlParserPos.ZERO)); + clusteredByArgs.add(SqlLiteral.createExactNumeric("3", SqlParserPos.ZERO)); + + Assert.assertEquals( + Arrays.asList("__time", "FOO", "BOO"), + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, sqlSelect) + ); + } + + @Test + public void testClusteredByOrdinalInvalidThrowsException() + { + final SqlNodeList selectArgs = new SqlNodeList(SqlParserPos.ZERO); + selectArgs.add(new SqlIdentifier("__time", new SqlParserPos(0, 1))); + selectArgs.add(new SqlIdentifier("FOO", new SqlParserPos(0, 2))); + selectArgs.add(new SqlIdentifier("BOO", new SqlParserPos(0, 3))); + + final SqlSelect sqlSelect = new SqlSelect( + SqlParserPos.ZERO, + null, + selectArgs, + null, + null, + null, + null, + null, + null, + null, + null + ); + + final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); + clusteredByArgs.add(new SqlIdentifier("__time", SqlParserPos.ZERO)); + clusteredByArgs.add(new SqlIdentifier("FOO", SqlParserPos.ZERO)); + clusteredByArgs.add(SqlLiteral.createExactNumeric("4", SqlParserPos.ZERO)); + + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, sqlSelect)), + DruidExceptionMatcher.invalidSqlInput().expectMessageIs( + "Ordinal[4] specified in the CLUSTERED BY clause is invalid. It must be between 1 and 3." + ) + ); + } + + + @Test + public void testClusteredByOrdinalsAndAliases() + { + // Construct the select source args + final SqlNodeList selectArgs = new SqlNodeList(SqlParserPos.ZERO); + selectArgs.add(new SqlIdentifier("__time", new SqlParserPos(0, 1))); + selectArgs.add(new SqlIdentifier("DIM3", new SqlParserPos(0, 2))); + + SqlBasicCall sqlBasicCall1 = new SqlBasicCall( + new SqlAsOperator(), + new SqlNode[]{ + new SqlIdentifier("DIM3", SqlParserPos.ZERO), + new SqlIdentifier("DIM3_ALIAS", SqlParserPos.ZERO) + }, + new SqlParserPos(0, 3) + ); + selectArgs.add(sqlBasicCall1); + + SqlBasicCall sqlBasicCall2 = new SqlBasicCall( + new SqlAsOperator(), + new SqlNode[]{ + new SqlIdentifier("FLOOR(__time)", SqlParserPos.ZERO), + new SqlIdentifier("floor_dim4_time", SqlParserPos.ZERO) + }, + new SqlParserPos(0, 4) + ); + selectArgs.add(sqlBasicCall2); + + selectArgs.add(new SqlIdentifier("DIM5", new SqlParserPos(0, 5))); + selectArgs.add(new SqlIdentifier("DIM6", new SqlParserPos(0, 6))); + + final SqlNodeList args3 = new SqlNodeList(SqlParserPos.ZERO); + args3.add(new SqlIdentifier("timestamps", SqlParserPos.ZERO)); + args3.add(SqlLiteral.createCharString("PT1H", SqlParserPos.ZERO)); + selectArgs.add(TimeFloorOperatorConversion.SQL_FUNCTION.createCall(args3)); + + final SqlSelect sqlSelect = new SqlSelect( + SqlParserPos.ZERO, + null, + selectArgs, + null, + null, + null, + null, + null, + null, + null, + null + ); + + // Construct the clustered by args + final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); + clusteredByArgs.add(SqlLiteral.createExactNumeric("3", SqlParserPos.ZERO)); + clusteredByArgs.add(SqlLiteral.createExactNumeric("4", SqlParserPos.ZERO)); + clusteredByArgs.add(SqlLiteral.createExactNumeric("5", SqlParserPos.ZERO)); + clusteredByArgs.add(SqlLiteral.createExactNumeric("7", SqlParserPos.ZERO)); + + Assert.assertEquals( + Arrays.asList("DIM3_ALIAS", "floor_dim4_time", "DIM5", "TIME_FLOOR(\"timestamps\", 'PT1H')"), + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, sqlSelect) + ); + } + + @Test + public void testSimpleClusteredByWithOrderBy() + { + final SqlNodeList selectArgs = new SqlNodeList(SqlParserPos.ZERO); + selectArgs.add(new SqlIdentifier("__time", new SqlParserPos(0, 1))); + selectArgs.add(new SqlIdentifier("FOO", new SqlParserPos(0, 2))); + selectArgs.add(new SqlIdentifier("BOO", new SqlParserPos(0, 3))); + + final SqlSelect sqlSelect = new SqlSelect( + SqlParserPos.ZERO, + null, + selectArgs, + null, + null, + null, + null, + null, + null, + null, + null + ); + + SqlNodeList orderList = new SqlNodeList(SqlParserPos.ZERO); + orderList.add(sqlSelect); + + SqlNode orderByNode = new SqlOrderBy(SqlParserPos.ZERO, sqlSelect, orderList, null, null); + + final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); + clusteredByArgs.add(new SqlIdentifier("__time", SqlParserPos.ZERO)); + clusteredByArgs.add(new SqlIdentifier("FOO", SqlParserPos.ZERO)); + clusteredByArgs.add(SqlLiteral.createExactNumeric("3", SqlParserPos.ZERO)); + + Assert.assertEquals( + Arrays.asList("__time", "FOO", "BOO"), + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, orderByNode) + ); + } + } + public static class ClusteredByColumnsValidationTest { /** diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlUnparseTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlUnparseTest.java index 20ea22c67a7..3f41bf7df35 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlUnparseTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlUnparseTest.java @@ -30,7 +30,7 @@ import java.io.StringReader; import static org.junit.Assert.assertEquals; /** - * A class containing unit tests for testing implmentations of {@link org.apache.calcite.sql.SqlNode#unparse(SqlWriter, int, int)} + * A class containing unit tests for testing implementations of {@link org.apache.calcite.sql.SqlNode#unparse(SqlWriter, int, int)} * in custom Druid SqlNode classes, like {@link DruidSqlInsert} and {@link DruidSqlReplace}. */ public class DruidSqlUnparseTest @@ -80,7 +80,6 @@ public class DruidSqlUnparseTest + "CLUSTERED BY \"dim1\""; DruidSqlParserImpl druidSqlParser = createTestParser(sqlQuery); DruidSqlReplace druidSqlReplace = (DruidSqlReplace) druidSqlParser.DruidSqlReplaceEof(); - druidSqlReplace.unparse(sqlWriter, 0, 0); assertEquals(prettySqlQuery, sqlWriter.toSqlString().getSql()); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/ExplainAttributesTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/ExplainAttributesTest.java index 0b3466634cd..67f97d64215 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/ExplainAttributesTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/ExplainAttributesTest.java @@ -21,29 +21,16 @@ package org.apache.druid.sql.calcite.planner; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNodeList; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.granularity.Granularities; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.mockito.Mockito; + +import java.util.Arrays; public class ExplainAttributesTest { private static final ObjectMapper DEFAULT_OBJECT_MAPPER = new DefaultObjectMapper(); - private static final SqlNode DATA_SOURCE = Mockito.mock(SqlNode.class); - private static final SqlNodeList CLUSTERED_BY = Mockito.mock(SqlNodeList.class); - private static final SqlNode TIME_CHUNKS = Mockito.mock(SqlNode.class); - - @Before - public void setup() - { - Mockito.when(DATA_SOURCE.toString()).thenReturn("foo"); - Mockito.when(CLUSTERED_BY.toString()).thenReturn("`bar`, `jazz`"); - Mockito.when(TIME_CHUNKS.toString()).thenReturn("ALL"); - } @Test public void testSimpleGetters() @@ -77,7 +64,7 @@ public class ExplainAttributesTest { ExplainAttributes insertAttributes = new ExplainAttributes( "INSERT", - DATA_SOURCE, + "foo", Granularities.DAY, null, null @@ -95,7 +82,7 @@ public class ExplainAttributesTest { ExplainAttributes insertAttributes = new ExplainAttributes( "INSERT", - DATA_SOURCE, + "foo", Granularities.ALL, null, null @@ -111,20 +98,72 @@ public class ExplainAttributesTest @Test public void testSerializeReplaceAttributes() throws JsonProcessingException { - ExplainAttributes replaceAttributes = new ExplainAttributes( + ExplainAttributes replaceAttributes1 = new ExplainAttributes( "REPLACE", - DATA_SOURCE, + "foo", Granularities.HOUR, - CLUSTERED_BY, - TIME_CHUNKS + null, + "ALL" ); - final String expectedAttributes = "{" + final String expectedAttributes1 = "{" + "\"statementType\":\"REPLACE\"," + "\"targetDataSource\":\"foo\"," + "\"partitionedBy\":\"HOUR\"," - + "\"clusteredBy\":\"`bar`, `jazz`\"," + "\"replaceTimeChunks\":\"ALL\"" + "}"; - Assert.assertEquals(expectedAttributes, DEFAULT_OBJECT_MAPPER.writeValueAsString(replaceAttributes)); + Assert.assertEquals(expectedAttributes1, DEFAULT_OBJECT_MAPPER.writeValueAsString(replaceAttributes1)); + + + ExplainAttributes replaceAttributes2 = new ExplainAttributes( + "REPLACE", + "foo", + Granularities.HOUR, + null, + "2019-08-25T02:00:00.000Z/2019-08-25T03:00:00.000Z" + ); + final String expectedAttributes2 = "{" + + "\"statementType\":\"REPLACE\"," + + "\"targetDataSource\":\"foo\"," + + "\"partitionedBy\":\"HOUR\"," + + "\"replaceTimeChunks\":\"2019-08-25T02:00:00.000Z/2019-08-25T03:00:00.000Z\"" + + "}"; + Assert.assertEquals(expectedAttributes2, DEFAULT_OBJECT_MAPPER.writeValueAsString(replaceAttributes2)); + } + + @Test + public void testSerializeReplaceWithClusteredByAttributes() throws JsonProcessingException + { + ExplainAttributes replaceAttributes1 = new ExplainAttributes( + "REPLACE", + "foo", + Granularities.HOUR, + Arrays.asList("foo", "CEIL(`f2`)"), + "ALL" + ); + final String expectedAttributes1 = "{" + + "\"statementType\":\"REPLACE\"," + + "\"targetDataSource\":\"foo\"," + + "\"partitionedBy\":\"HOUR\"," + + "\"clusteredBy\":[\"foo\",\"CEIL(`f2`)\"]," + + "\"replaceTimeChunks\":\"ALL\"" + + "}"; + Assert.assertEquals(expectedAttributes1, DEFAULT_OBJECT_MAPPER.writeValueAsString(replaceAttributes1)); + + + ExplainAttributes replaceAttributes2 = new ExplainAttributes( + "REPLACE", + "foo", + Granularities.HOUR, + Arrays.asList("foo", "boo"), + "2019-08-25T02:00:00.000Z/2019-08-25T03:00:00.000Z" + ); + final String expectedAttributes2 = "{" + + "\"statementType\":\"REPLACE\"," + + "\"targetDataSource\":\"foo\"," + + "\"partitionedBy\":\"HOUR\"," + + "\"clusteredBy\":[\"foo\",\"boo\"]," + + "\"replaceTimeChunks\":\"2019-08-25T02:00:00.000Z/2019-08-25T03:00:00.000Z\"" + + "}"; + Assert.assertEquals(expectedAttributes2, DEFAULT_OBJECT_MAPPER.writeValueAsString(replaceAttributes2)); } } From 903addf7c281099cb3fd510c0ebebfa5d62c5584 Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Mon, 26 Jun 2023 20:03:08 -0700 Subject: [PATCH 33/74] Make agg and scalar routines test to depend on specific routine names. (#14482) --- .../org/apache/druid/sql/calcite/CalciteQueryTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index fcf2c9dbb2b..de6697388fc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -385,10 +385,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest "SELECT\n" + " COUNT(*)\n" + "FROM INFORMATION_SCHEMA.ROUTINES\n" - + "WHERE IS_AGGREGATOR = 'YES'", + + "WHERE IS_AGGREGATOR = 'YES' AND ROUTINE_NAME = 'COUNT'", ImmutableList.of(), ImmutableList.of( - new Object[]{30L} + new Object[]{1L} ) ); } @@ -401,10 +401,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest "SELECT\n" + " COUNT(*)\n" + "FROM INFORMATION_SCHEMA.ROUTINES\n" - + "WHERE IS_AGGREGATOR = 'NO'", + + "WHERE IS_AGGREGATOR = 'NO' AND ROUTINE_NAME = 'CEIL'", ImmutableList.of(), ImmutableList.of( - new Object[]{152L} + new Object[]{1L} ) ); } From f546cd64a9640ed14c9c0f44481678d39530e3b5 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Tue, 27 Jun 2023 09:25:32 +0530 Subject: [PATCH 34/74] MSQ: Ensure that the allocated segment aligns with the requested granularity (#14475) Changes: - Throw an `InsertCannotAllocateSegmentFault` if the allocated segment is not aligned with the requested granularity. - Tests to verify new behaviour --- docs/multi-stage-query/reference.md | 2 +- .../apache/druid/msq/exec/ControllerImpl.java | 17 +++- .../InsertCannotAllocateSegmentFault.java | 70 ++++++++++++- .../apache/druid/msq/util/IntervalUtils.java | 22 ++++- .../apache/druid/msq/exec/MSQFaultsTest.java | 54 +++++++++- .../msq/indexing/error/MSQFaultSerdeTest.java | 7 +- .../druid/msq/util/IntervalUtilsTest.java | 99 +++++++++++++++++++ .../util/common/granularity/Granularity.java | 3 +- 8 files changed, 262 insertions(+), 12 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index d56d86964f1..ec6a5b1543d 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -423,7 +423,7 @@ The following table describes error codes you may encounter in the `multiStageQu | `CannotParseExternalData` | A worker task could not parse data from an external datasource. | `errorMessage`: More details on why parsing failed. | | `ColumnNameRestricted` | The query uses a restricted column name. | `columnName`: The restricted column name. | | `ColumnTypeNotSupported` | The column type is not supported. This can be because:

  • Support for writing or reading from a particular column type is not supported.
  • The query attempted to use a column type that is not supported by the frame format. This occurs with ARRAY types, which are not yet implemented for frames.
| `columnName`: The column name with an unsupported type.

`columnType`: The unknown column type. | -| `InsertCannotAllocateSegment` | The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts:

  • Attempting to mix different granularities in the same intervals of the same datasource.
  • Prior ingestions that used non-extendable shard specs.
| `dataSource`

`interval`: The interval for the attempted new segment allocation. | +| `InsertCannotAllocateSegment` | The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts:

  • Attempting to mix different granularities in the same intervals of the same datasource.
  • Prior ingestions that used non-extendable shard specs.


Use REPLACE to overwrite the existing data or if the error contains the `allocatedInterval` then alternatively rerun the INSERT job with the mentioned granularity to append to existing data. Note that it might not always be possible to append to the existing data using INSERT and can only be done if `allocatedInterval` is present. | `dataSource`

`interval`: The interval for the attempted new segment allocation.

`allocatedInterval`: The incorrect interval allocated by the overlord. It can be null | | `InsertCannotBeEmpty` | An INSERT or REPLACE query did not generate any output rows in a situation where output rows are required for success. This can happen for INSERT or REPLACE queries with `PARTITIONED BY` set to something other than `ALL` or `ALL TIME`. | `dataSource` | | `InsertLockPreempted` | An INSERT or REPLACE query was canceled by a higher-priority ingestion job, such as a real-time ingestion task. | | | `InsertTimeNull` | An INSERT or REPLACE query encountered a null timestamp in the `__time` field.

This can happen due to using an expression like `TIME_PARSE(timestamp) AS __time` with a timestamp that cannot be parsed. ([`TIME_PARSE`](../querying/sql-scalar.md#date-and-time-functions) returns null when it cannot parse a timestamp.) In this case, try parsing your timestamps using a different function or pattern. Or, if your timestamps may genuinely be null, consider using [`COALESCE`](../querying/sql-scalar.md#other-scalar-functions) to provide a default value. One option is [`CURRENT_TIMESTAMP`](../querying/sql-scalar.md#date-and-time-functions), which represents the start time of the job.| diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index dd163d406d3..db5e6997179 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -940,7 +940,22 @@ public class ControllerImpl implements Controller throw new MSQException( new InsertCannotAllocateSegmentFault( task.getDataSource(), - segmentGranularity.bucket(timestamp) + segmentGranularity.bucket(timestamp), + null + ) + ); + } + + // Even if allocation isn't null, the overlord makes the best effort job of allocating a segment with the given + // segmentGranularity. This is commonly seen in case when there is already a coarser segment in the interval where + // the requested segment is present and that segment completely overlaps the request. Throw an error if the interval + // doesn't match the granularity requested + if (!IntervalUtils.isAligned(allocation.getInterval(), segmentGranularity)) { + throw new MSQException( + new InsertCannotAllocateSegmentFault( + task.getDataSource(), + segmentGranularity.bucket(timestamp), + allocation.getInterval() ) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotAllocateSegmentFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotAllocateSegmentFault.java index 403af37d9bf..f632ae67736 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotAllocateSegmentFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotAllocateSegmentFault.java @@ -23,8 +23,12 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.DurationGranularity; +import org.apache.druid.java.util.common.granularity.GranularityType; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Objects; @JsonTypeName(InsertCannotAllocateSegmentFault.CODE) @@ -35,15 +39,20 @@ public class InsertCannotAllocateSegmentFault extends BaseMSQFault private final String dataSource; private final Interval interval; + @Nullable + private final Interval allocatedInterval; + @JsonCreator public InsertCannotAllocateSegmentFault( @JsonProperty("dataSource") final String dataSource, - @JsonProperty("interval") final Interval interval + @JsonProperty("interval") final Interval interval, + @Nullable @JsonProperty("allocatedInterval") final Interval allocatedInterval ) { - super(CODE, "Cannot allocate segment for dataSource [%s], interval [%s]", dataSource, interval); + super(CODE, getErrorMessage(dataSource, interval, allocatedInterval)); this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.interval = Preconditions.checkNotNull(interval, "interval"); + this.allocatedInterval = allocatedInterval; } @JsonProperty @@ -58,6 +67,57 @@ public class InsertCannotAllocateSegmentFault extends BaseMSQFault return interval; } + @Nullable + @JsonProperty + public Interval getAllocatedInterval() + { + return allocatedInterval; + } + + private static String getErrorMessage( + final String dataSource, + final Interval interval, + @Nullable final Interval allocatedInterval + ) + { + String errorMessage; + if (allocatedInterval == null) { + errorMessage = StringUtils.format( + "Cannot allocate segment for dataSource [%s], interval [%s]. This can happen if the prior ingestion " + + "uses non-extendable shard specs or if the partitioned by granularity is different from the granularity of the " + + "pre-existing segments. Check the granularities of the pre-existing segments or re-run the ingestion with REPLACE " + + "to overwrite over the existing data", + dataSource, + interval + ); + } else { + errorMessage = StringUtils.format( + "Requested segment for dataSource [%s], interval [%s], but got [%s] interval instead. " + + "This happens when an overlapping segment is already present with a coarser granularity for the requested interval. " + + "Either set the partition granularity for the INSERT to [%s] to append to existing data or use REPLACE to " + + "overwrite over the pre-existing segment", + dataSource, + interval, + allocatedInterval, + convertIntervalToGranularityString(allocatedInterval) + ); + } + return errorMessage; + } + + /** + * Converts the given interval to a string representing the granularity which is more user-friendly. + */ + private static String convertIntervalToGranularityString(final Interval interval) + { + try { + return GranularityType.fromPeriod(interval.toPeriod()).name(); + } + catch (Exception e) { + return new DurationGranularity(interval.toDurationMillis(), null).toString(); + } + } + @Override public boolean equals(Object o) { @@ -71,12 +131,14 @@ public class InsertCannotAllocateSegmentFault extends BaseMSQFault return false; } InsertCannotAllocateSegmentFault that = (InsertCannotAllocateSegmentFault) o; - return Objects.equals(dataSource, that.dataSource) && Objects.equals(interval, that.interval); + return Objects.equals(dataSource, that.dataSource) + && Objects.equals(interval, that.interval) + && Objects.equals(allocatedInterval, that.allocatedInterval); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), dataSource, interval); + return Objects.hash(super.hashCode(), dataSource, interval, allocatedInterval); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/IntervalUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/IntervalUtils.java index 43a844b5a6f..328a0c0b74d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/IntervalUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/IntervalUtils.java @@ -19,13 +19,16 @@ package org.apache.druid.msq.util; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.AllGranularity; +import org.apache.druid.java.util.common.granularity.Granularity; import org.joda.time.Interval; import java.util.ArrayList; import java.util.List; /** - * Things that would make sense in {@link org.apache.druid.java.util.common.Intervals} if this were not an extension. + * Things that would make sense in {@link Intervals} if this were not an extension. */ public class IntervalUtils { @@ -61,4 +64,21 @@ public class IntervalUtils return retVal; } + + /** + * This method checks if the provided interval is aligned by the granularity or is an instance of {@link Intervals#ETERNITY} + * This is used to check if the granularity allocation made by the overlord is the same as the one requested in the + * SQL query + */ + public static boolean isAligned( + final Interval interval, + final Granularity granularity + ) + { + // AllGranularity needs special handling since AllGranularity#bucketStart always returns false + if (granularity instanceof AllGranularity) { + return Intervals.isEternity(interval); + } + return granularity.isAligned(interval); + } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java index 646286acaf5..cfcc1073331 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java @@ -36,6 +36,8 @@ import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestFileUtils; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.LinearShardSpec; import org.junit.Test; import org.mockito.Mockito; @@ -51,7 +53,7 @@ import static org.mockito.ArgumentMatchers.isA; public class MSQFaultsTest extends MSQTestBase { @Test - public void testInsertCannotAllocateSegmentFault() + public void testInsertCannotAllocateSegmentFaultWhenNullAllocation() { RowSignature rowSignature = RowSignature.builder() .add("__time", ColumnType.LONG) @@ -62,18 +64,64 @@ public class MSQFaultsTest extends MSQTestBase Mockito.doReturn(null).when(testTaskActionClient).submit(isA(SegmentAllocateAction.class)); testIngestQuery().setSql( - "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null and __time >= TIMESTAMP '2000-01-02 00:00:00' and __time < TIMESTAMP '2000-01-03 00:00:00' group by 1, 2 PARTITIONED by day clustered by dim1") + "insert into foo1" + + " select __time, dim1 , count(*) as cnt" + + " from foo" + + " where dim1 is not null and __time >= TIMESTAMP '2000-01-02 00:00:00' and __time < TIMESTAMP '2000-01-03 00:00:00'" + + " group by 1, 2" + + " PARTITIONED by day" + + " clustered by dim1" + ) .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setExpectedMSQFault( new InsertCannotAllocateSegmentFault( "foo1", - Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z") + Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z"), + null ) ) .verifyResults(); } + @Test + public void testInsertCannotAllocateSegmentFaultWhenInvalidAllocation() + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG).build(); + + // If there is some problem allocating the segment,task action client will return a null value. + Mockito.doReturn(new SegmentIdWithShardSpec( + "foo1", + Intervals.of("2000-01-01/2000-02-01"), + "test", + new LinearShardSpec(2) + )).when(testTaskActionClient).submit(isA(SegmentAllocateAction.class)); + + testIngestQuery().setSql( + "insert into foo1" + + " select __time, dim1 , count(*) as cnt" + + " from foo" + + " where dim1 is not null and __time >= TIMESTAMP '2000-01-02 00:00:00' and __time < TIMESTAMP '2000-01-03 00:00:00'" + + " group by 1, 2" + + " PARTITIONED by day" + + " clustered by dim1" + ) + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .setExpectedMSQFault( + new InsertCannotAllocateSegmentFault( + "foo1", + Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z"), + Intervals.of("2000-01-01T00:00:00.000Z/2000-02-01T00:00:00.000Z") + ) + ) + .verifyResults(); + } + + @Test public void testInsertCannotBeEmptyFault() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index c2ebb2d25ac..bcd0de1ef68 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -54,7 +54,12 @@ public class MSQFaultSerdeTest assertFaultSerde(new ColumnTypeNotSupportedFault("the column", null)); assertFaultSerde(new ColumnTypeNotSupportedFault("the column", ColumnType.STRING_ARRAY)); assertFaultSerde(new ColumnNameRestrictedFault("the column")); - assertFaultSerde(new InsertCannotAllocateSegmentFault("the datasource", Intervals.ETERNITY)); + assertFaultSerde(new InsertCannotAllocateSegmentFault("the datasource", Intervals.ETERNITY, null)); + assertFaultSerde(new InsertCannotAllocateSegmentFault( + "the datasource", + Intervals.of("2000-01-01/2002-01-01"), + Intervals.ETERNITY + )); assertFaultSerde(new InsertCannotBeEmptyFault("the datasource")); assertFaultSerde(InsertLockPreemptedFault.INSTANCE); assertFaultSerde(InsertTimeNullFault.INSTANCE); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/IntervalUtilsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/IntervalUtilsTest.java index 6dd47313e7b..03371f8b90f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/IntervalUtilsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/IntervalUtilsTest.java @@ -19,8 +19,12 @@ package org.apache.druid.msq.util; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.joda.time.Interval; +import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; @@ -84,6 +88,101 @@ public class IntervalUtilsTest ); } + @Test + public void test_doesIntervalMatchesGranularity_withStandardGranularities() + { + + Assert.assertTrue( + IntervalUtils.isAligned(Intervals.ETERNITY, Granularities.ALL) + ); + + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("2000-01-01/2001-01-01"), Granularities.YEAR + ) + ); + + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("2000-01-01/2000-04-01"), Granularities.QUARTER + ) + ); + + + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("2000-01-01/2000-02-01"), Granularities.MONTH + ) + ); + + // With the way WEEK granularities work, this needs to be aligned to an actual week + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("1999-12-27/2000-01-03"), Granularities.WEEK + ) + ); + + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("2000-01-01/2000-01-02"), Granularities.DAY + ) + ); + + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("2000-01-01T00:00:00.000/2000-01-01T08:00:00.000"), Granularities.EIGHT_HOUR + ) + ); + + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("2000-01-01T00:00:00.000/2000-01-01T01:00:00.000"), Granularities.HOUR + ) + ); + + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("2000-01-01T00:00:00.000/2000-01-01T00:01:00.000"), Granularities.MINUTE + ) + ); + + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("2000-01-01T00:00:00.000/2000-01-01T00:00:01.000"), Granularities.SECOND + ) + ); + + Assert.assertFalse( + IntervalUtils.isAligned( + Intervals.of("2000-01-01/2002-01-01"), Granularities.YEAR + ) + ); + + Assert.assertFalse( + IntervalUtils.isAligned( + Intervals.of("2000-01-01/2002-01-08"), Granularities.YEAR + ) + ); + } + + @Test + public void test_doesIntervalMatchesGranularity_withPeriodGranularity() + { + Assert.assertTrue( + IntervalUtils.isAligned( + Intervals.of("2000-01-01/2000-01-04"), + new PeriodGranularity(new Period("P3D"), DateTimes.of("2000-01-01"), null) + ) + ); + + Assert.assertFalse( + IntervalUtils.isAligned( + Intervals.of("2000-01-01/2000-01-04"), + new PeriodGranularity(new Period("P3D"), DateTimes.of("2000-01-02"), null) + ) + ); + } + public static List intervals(final String... intervals) { return Arrays.stream(intervals).map(Intervals::of).collect(Collectors.toList()); diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java index 812538ae6e0..6f5d9fb61e1 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java @@ -145,7 +145,8 @@ public abstract class Granularity implements Cacheable public abstract DateTime toDate(String filePath, Formatter formatter); /** - * Return true if time chunks populated by this granularity includes the given interval time chunk. + * Return true only if the time chunks populated by this granularity includes the given interval time chunk. The + * interval must fit exactly into the scheme of the granularity for this to return true */ public abstract boolean isAligned(Interval interval); From 6ba10c8b6caf1de5afd782ba70edda406466c892 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 26 Jun 2023 21:02:44 -0700 Subject: [PATCH 35/74] fix bug with json_value expression array extraction (#14461) --- .../expression/NestedDataExpressions.java | 24 ++-- .../druid/segment/AutoTypeColumnIndexer.java | 2 +- .../druid/segment/column/TypeSignature.java | 6 + .../nested/NestedDataColumnSerializer.java | 4 +- .../nested/NestedDataColumnSerializerV4.java | 2 +- .../segment/nested/NestedPathFinder.java | 106 +++++++----------- .../virtual/NestedFieldVirtualColumn.java | 22 +++- .../expression/NestedDataExpressionsTest.java | 73 +++++++++++- .../segment/nested/NestedPathFinderTest.java | 12 -- .../NestedDataOperatorConversions.java | 2 +- .../calcite/CalciteNestedDataQueryTest.java | 2 +- 11 files changed, 155 insertions(+), 100 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java index 7e62207111b..57b81116fa8 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java +++ b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java @@ -349,10 +349,14 @@ public class NestedDataExpressions @Override public ExprEval eval(ObjectBinding bindings) { - ExprEval input = args.get(0).eval(bindings); - return ExprEval.bestEffortOf( - NestedPathFinder.findLiteral(unwrap(input), parts) - ).castTo(castTo); + final ExprEval input = args.get(0).eval(bindings); + final ExprEval valAtPath = ExprEval.bestEffortOf( + NestedPathFinder.find(unwrap(input), parts) + ); + if (valAtPath.type().isPrimitive() || valAtPath.type().isPrimitiveArray()) { + return valAtPath.castTo(castTo); + } + return ExprEval.ofType(castTo, null); } @Override @@ -382,10 +386,14 @@ public class NestedDataExpressions @Override public ExprEval eval(ObjectBinding bindings) { - ExprEval input = args.get(0).eval(bindings); - return ExprEval.bestEffortOf( - NestedPathFinder.findLiteral(unwrap(input), parts) + final ExprEval input = args.get(0).eval(bindings); + final ExprEval valAtPath = ExprEval.bestEffortOf( + NestedPathFinder.find(unwrap(input), parts) ); + if (valAtPath.type().isPrimitive() || valAtPath.type().isPrimitiveArray()) { + return valAtPath; + } + return ExprEval.of(null); } @Override @@ -489,7 +497,7 @@ public class NestedDataExpressions { // we only want to return a non-null value here if the value is an array of primitive values ExprEval eval = ExprEval.bestEffortArray(array); - if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) { + if (eval.type().isPrimitiveArray()) { return ProcessedValue.NULL_LITERAL; } return null; diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java index c52c15d0634..3ce8b5f4ba6 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java @@ -104,7 +104,7 @@ public class AutoTypeColumnIndexer implements DimensionIndexer eval = ExprEval.bestEffortArray(array); - if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) { + if (eval.type().isPrimitiveArray()) { final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath); FieldIndexer fieldIndexer = fieldIndexers.get(fieldName); if (fieldIndexer == null) { diff --git a/processing/src/main/java/org/apache/druid/segment/column/TypeSignature.java b/processing/src/main/java/org/apache/druid/segment/column/TypeSignature.java index 0c35a2934f0..330a2bb9d5d 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/TypeSignature.java +++ b/processing/src/main/java/org/apache/druid/segment/column/TypeSignature.java @@ -167,6 +167,12 @@ public interface TypeSignature return getType().isArray(); } + @JsonIgnore + default boolean isPrimitiveArray() + { + return getType().isArray() && getElementType() != null && getElementType().isPrimitive(); + } + /** * Convert a {@link TypeSignature} into a simple string. This value can be converted back into a {@link TypeSignature} * with {@link Types#fromString(TypeFactory, String)}. diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java index 020298b1b35..3e49edc32a0 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java @@ -98,7 +98,7 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ if (writer != null) { try { final ExprEval eval = ExprEval.bestEffortOf(fieldValue); - if (eval.type().isPrimitive() || (eval.type().isArray() && eval.type().getElementType().isPrimitive())) { + if (eval.type().isPrimitive() || eval.type().isPrimitiveArray()) { writer.addValue(rowCount, eval.value()); } else { // behave consistently with nested column indexer, which defaults to string @@ -122,7 +122,7 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ ) { final ExprEval eval = ExprEval.bestEffortArray(array); - if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) { + if (eval.type().isPrimitiveArray()) { final GlobalDictionaryEncodedFieldColumnWriter writer = fieldWriters.get( NestedPathFinder.toNormalizedJsonPath(fieldPath) ); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializerV4.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializerV4.java index 034c7bc64f5..d33d5f68f47 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializerV4.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializerV4.java @@ -90,7 +90,7 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer eval = ExprEval.bestEffortOf(fieldValue); - if (eval.type().isPrimitive() || (eval.type().isArray() && eval.type().getElementType().isPrimitive())) { + if (eval.type().isPrimitive() || eval.type().isPrimitiveArray()) { writer.addValue(rowCount, eval.value()); } else { // behave consistently with nested column indexer, which defaults to string diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedPathFinder.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedPathFinder.java index 5d0596a6cc8..2fc52f88df6 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedPathFinder.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedPathFinder.java @@ -34,6 +34,45 @@ public class NestedPathFinder public static final String JSON_PATH_ROOT = "$"; public static final String JQ_PATH_ROOT = "."; + /** + * Dig through a thing to find stuff + */ + @Nullable + public static Object find(@Nullable Object data, List path) + { + Object currentObject = data; + for (NestedPathPart pathPart : path) { + Object objectAtPath = pathPart.find(currentObject); + if (objectAtPath == null) { + return null; + } + currentObject = objectAtPath; + } + return currentObject; + } + + /** + * find the list of 'keys' at some path. + * - if the thing at a path is an object (map), return the fields + * - if the thing at a path is an array (list or array), return the 0 based element numbers + * - if the thing is a simple value, return null + */ + @Nullable + public static Object[] findKeys(@Nullable Object data, List path) + { + Object currentObject = find(data, path); + if (currentObject instanceof Map) { + return ((Map) currentObject).keySet().toArray(); + } + if (currentObject instanceof List) { + return IntStream.range(0, ((List) currentObject).size()).mapToObj(Integer::toString).toArray(); + } + if (currentObject instanceof Object[]) { + return IntStream.range(0, ((Object[]) currentObject).length).mapToObj(Integer::toString).toArray(); + } + return null; + } + public static String toNormalizedJsonPath(List paths) { if (paths.isEmpty()) { @@ -305,71 +344,4 @@ public class NestedPathFinder { throw InvalidInput.exception("JSONPath [%s] is invalid, %s", path, StringUtils.format(message, args)); } - - /** - * Dig through a thing to find stuff, if that stuff is a not nested itself - */ - @Nullable - public static String findStringLiteral(@Nullable Object data, List path) - { - Object currentObject = find(data, path); - if (currentObject instanceof Map || currentObject instanceof List || currentObject instanceof Object[]) { - return null; - } else { - // a literal of some sort, huzzah! - if (currentObject == null) { - return null; - } - return String.valueOf(currentObject); - } - } - - @Nullable - public static Object findLiteral(@Nullable Object data, List path) - { - Object currentObject = find(data, path); - if (currentObject instanceof Map || currentObject instanceof List || currentObject instanceof Object[]) { - return null; - } else { - // a literal of some sort, huzzah! - if (currentObject == null) { - return null; - } - return currentObject; - } - } - - @Nullable - public static Object[] findKeys(@Nullable Object data, List path) - { - Object currentObject = find(data, path); - if (currentObject instanceof Map) { - return ((Map) currentObject).keySet().toArray(); - } - if (currentObject instanceof List) { - return IntStream.range(0, ((List) currentObject).size()).mapToObj(Integer::toString).toArray(); - } - if (currentObject instanceof Object[]) { - return IntStream.range(0, ((Object[]) currentObject).length).mapToObj(Integer::toString).toArray(); - } - return null; - } - - /** - * Dig through a thing to find stuff - */ - @Nullable - public static Object find(@Nullable Object data, List path) - { - Object currentObject = data; - for (NestedPathPart pathPart : path) { - Object objectAtPath = pathPart.find(currentObject); - if (objectAtPath == null) { - return null; - } - currentObject = objectAtPath; - } - return currentObject; - } - } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java index 486448c2e91..02c3d879e83 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java @@ -169,7 +169,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn String columnName, String path, String outputName, - ColumnType expectedType + @Nullable ColumnType expectedType ) { this(columnName, outputName, expectedType, null, null, path, false); @@ -1266,8 +1266,8 @@ public class NestedFieldVirtualColumn implements VirtualColumn } /** - * Process the "raw" data to extract literals with {@link NestedPathFinder#findLiteral(Object, List)}. Like - * {@link RawFieldColumnSelector} but only literals and does not wrap the results in {@link StructuredData}. + * Process the "raw" data to extract non-complex values. Like {@link RawFieldColumnSelector} but does not return + * complex nested objects and does not wrap the results in {@link StructuredData}. *

* This is used as a selector on realtime data when the native field columns are not available. */ @@ -1312,7 +1312,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn @Override public boolean isNull() { - Object o = getObject(); + final Object o = getObject(); return !(o instanceof Number || (o instanceof String && Doubles.tryParse((String) o) != null)); } @@ -1320,8 +1320,18 @@ public class NestedFieldVirtualColumn implements VirtualColumn @Override public Object getObject() { - StructuredData data = StructuredData.wrap(baseSelector.getObject()); - return NestedPathFinder.findLiteral(data == null ? null : data.getValue(), parts); + final StructuredData data = StructuredData.wrap(baseSelector.getObject()); + if (data == null) { + return null; + } + + final Object valAtPath = NestedPathFinder.find(data.getValue(), parts); + final ExprEval eval = ExprEval.bestEffortOf(valAtPath); + if (eval.type().isPrimitive() || eval.type().isPrimitiveArray()) { + return eval.valueOrDefault(); + } + // not a primitive value, return null; + return null; } @Override diff --git a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java index afb6c9298c8..06726289ce6 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java @@ -36,6 +36,7 @@ import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; import java.util.Map; public class NestedDataExpressionsTest extends InitializedNullHandlingTest @@ -64,11 +65,27 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest "y", ImmutableMap.of("a", "hello", "b", "world") ); + private static final Map NESTERER = ImmutableMap.of( + "x", + ImmutableMap.of( + "a1", Arrays.asList(1, null, 3), + "a2", Arrays.asList(1.1, null, 3.3), + "a3", Arrays.asList("a", null, "b", "100") + ), + "y", + ImmutableList.of( + ImmutableMap.of("x", 1L, "y", 1.1), + ImmutableMap.of("x", 2L, "y", 2.2), + ImmutableMap.of("x", 3L, "y", 3.3) + ) + ); + Expr.ObjectBinding inputBindings = InputBindings.forInputSuppliers( new ImmutableMap.Builder() .put("nest", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> NEST)) .put("nestWrapped", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> new StructuredData(NEST))) .put("nester", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> NESTER)) + .put("nesterer", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> NESTERER)) .put("string", InputBindings.inputSupplier(ExpressionType.STRING, () -> "abcdef")) .put("long", InputBindings.inputSupplier(ExpressionType.LONG, () -> 1234L)) .put("double", InputBindings.inputSupplier(ExpressionType.DOUBLE, () -> 1.234)) @@ -144,7 +161,8 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest expr = Parser.parse("json_value(nester, '$.x')", MACRO_TABLE); eval = expr.eval(inputBindings); - Assert.assertNull(eval.value()); + Assert.assertArrayEquals(new Object[]{"a", "b", "c"}, (Object[]) eval.value()); + Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type()); expr = Parser.parse("json_value(nester, '$.x[1]')", MACRO_TABLE); eval = expr.eval(inputBindings); @@ -201,6 +219,59 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest eval = expr.eval(inputBindings); Assert.assertEquals("100", eval.value()); Assert.assertEquals(ExpressionType.STRING, eval.type()); + + expr = Parser.parse("json_value(nesterer, '$.x.a1')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertArrayEquals(new Object[]{1L, null, 3L}, (Object[]) eval.value()); + Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type()); + + expr = Parser.parse("json_value(nesterer, '$.x.a1', 'ARRAY')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertArrayEquals(new Object[]{"1", null, "3"}, (Object[]) eval.value()); + Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type()); + + expr = Parser.parse("json_value(nesterer, '$.x.a1', 'ARRAY')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertArrayEquals(new Object[]{1.0, null, 3.0}, (Object[]) eval.value()); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type()); + + expr = Parser.parse("json_value(nesterer, '$.x.a2')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertArrayEquals(new Object[]{1.1, null, 3.3}, (Object[]) eval.value()); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, eval.type()); + + expr = Parser.parse("json_value(nesterer, '$.x.a2', 'ARRAY')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertArrayEquals(new Object[]{1L, null, 3L}, (Object[]) eval.value()); + Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type()); + + expr = Parser.parse("json_value(nesterer, '$.x.a2', 'ARRAY')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertArrayEquals(new Object[]{"1.1", null, "3.3"}, (Object[]) eval.value()); + Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type()); + + expr = Parser.parse("json_value(nesterer, '$.x.a3')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertArrayEquals(new Object[]{"a", null, "b", "100"}, (Object[]) eval.value()); + Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type()); + + expr = Parser.parse("json_value(nesterer, '$.x.a3', 'ARRAY')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertArrayEquals( + new Object[]{null, null, null, 100L}, + (Object[]) eval.value() + ); + Assert.assertEquals(ExpressionType.LONG_ARRAY, eval.type()); + + // arrays of objects are not primitive + expr = Parser.parse("json_value(nesterer, '$.y')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertNull(eval.value()); + + expr = Parser.parse("json_value(json_object('k1', array(1,2,3), 'k2', array('a', 'b', 'c')), '$.k1', 'ARRAY')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertArrayEquals(new Object[]{"1", "2", "3"}, (Object[]) eval.value()); + Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedPathFinderTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedPathFinderTest.java index b22131833c6..5f0a328f687 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedPathFinderTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedPathFinderTest.java @@ -417,51 +417,39 @@ public class NestedPathFinderTest pathParts = NestedPathFinder.parseJqPath("."); Assert.assertEquals(NESTER, NestedPathFinder.find(NESTER, pathParts)); - Assert.assertNull(NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".z"); Assert.assertEquals("foo", NestedPathFinder.find(NESTER, pathParts)); - Assert.assertEquals("foo", NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".x"); Assert.assertEquals(NESTER.get("x"), NestedPathFinder.find(NESTER, pathParts)); - Assert.assertNull(NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".x[1]"); Assert.assertEquals("b", NestedPathFinder.find(NESTER, pathParts)); - Assert.assertEquals("b", NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".x[-1]"); Assert.assertEquals("c", NestedPathFinder.find(NESTER, pathParts)); - Assert.assertEquals("c", NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".x[-2]"); Assert.assertEquals("b", NestedPathFinder.find(NESTER, pathParts)); - Assert.assertEquals("b", NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".x[-4]"); Assert.assertNull(NestedPathFinder.find(NESTER, pathParts)); - Assert.assertNull(NestedPathFinder.findStringLiteral(NESTER, pathParts)); // nonexistent pathParts = NestedPathFinder.parseJqPath(".x[1].y.z"); Assert.assertNull(NestedPathFinder.find(NESTER, pathParts)); - Assert.assertNull(NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".y.a"); Assert.assertEquals("hello", NestedPathFinder.find(NESTER, pathParts)); - Assert.assertEquals("hello", NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".y[1]"); Assert.assertNull(NestedPathFinder.find(NESTER, pathParts)); - Assert.assertNull(NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".\"[sneaky]\""); Assert.assertEquals("bar", NestedPathFinder.find(NESTER, pathParts)); - Assert.assertEquals("bar", NestedPathFinder.findStringLiteral(NESTER, pathParts)); pathParts = NestedPathFinder.parseJqPath(".\"[also_sneaky]\"[1].c"); Assert.assertEquals("z", NestedPathFinder.find(NESTER, pathParts)); - Assert.assertEquals("z", NestedPathFinder.findStringLiteral(NESTER, pathParts)); } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java index c5503f7eb85..949b8b8879d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java @@ -686,7 +686,7 @@ public class NestedDataOperatorConversions (name, outputType, expression, macroTable) -> new NestedFieldVirtualColumn( druidExpressions.get(0).getDirectColumn(), name, - outputType, + null, parts, false, null, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 83861485960..628d7dc3246 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -440,7 +440,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - new NestedFieldVirtualColumn("nest", "$.x", "v0", ColumnType.STRING) + new NestedFieldVirtualColumn("nest", "$.x", "v0", null) ) .setDimensions( dimensions( From 2f0a43790c22318b9c48ed0f38db7350fef267ef Mon Sep 17 00:00:00 2001 From: imply-cheddar <86940447+imply-cheddar@users.noreply.github.com> Date: Tue, 27 Jun 2023 13:45:29 +0900 Subject: [PATCH 36/74] Make GuavaUtilsTest use less CPU (#14487) --- .../druid/common/guava/GuavaUtilsTest.java | 43 +++++++++++-------- 1 file changed, 24 insertions(+), 19 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/common/guava/GuavaUtilsTest.java b/processing/src/test/java/org/apache/druid/common/guava/GuavaUtilsTest.java index 3fcfb72cfd4..317cb350f97 100644 --- a/processing/src/test/java/org/apache/druid/common/guava/GuavaUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/common/guava/GuavaUtilsTest.java @@ -31,9 +31,11 @@ import org.junit.Test; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; public class GuavaUtilsTest @@ -73,23 +75,23 @@ public class GuavaUtilsTest int tasks = 3; ExecutorService service = Execs.multiThreaded(tasks, "GuavaUtilsTest-%d"); ListeningExecutorService exc = MoreExecutors.listeningDecorator(service); - AtomicInteger index = new AtomicInteger(0); //a flag what time to throw exception. - AtomicBoolean active = new AtomicBoolean(false); + AtomicBoolean someoneFailed = new AtomicBoolean(false); + List latches = new ArrayList<>(tasks); Function>> function = (taskCount) -> { List> futures = new ArrayList<>(); for (int i = 0; i < taskCount; i++) { + final CountDownLatch latch = new CountDownLatch(1); + latches.add(latch); ListenableFuture future = exc.submit(new Callable() { @Override - public Object call() throws RuntimeException + public Object call() throws RuntimeException, InterruptedException { - int internalIndex = index.incrementAndGet(); - while (true) { - if (internalIndex == taskCount && active.get()) { - //here we simulate occurs exception in some one future. - throw new RuntimeException("A big bug"); - } + latch.await(60, TimeUnit.SECONDS); + if (someoneFailed.compareAndSet(false, true)) { + throw new RuntimeException("This exception simulates an error"); } + return null; } }); futures.add(future); @@ -99,17 +101,20 @@ public class GuavaUtilsTest List> futures = function.apply(tasks); Assert.assertEquals(tasks, futures.stream().filter(f -> !f.isDone()).count()); - //here we make one of task throw exception. - active.set(true); + // "release" the last tasks, which will cause it to fail as someoneFailed will still be false + latches.get(tasks - 1).countDown(); ListenableFuture> future = Futures.allAsList(futures); - try { - future.get(); - } - catch (Exception e) { - Assert.assertEquals("java.lang.RuntimeException: A big bug", e.getMessage()); - GuavaUtils.cancelAll(true, future, futures); - Assert.assertEquals(0, futures.stream().filter(f -> !f.isDone()).count()); + + ExecutionException thrown = Assert.assertThrows( + ExecutionException.class, + future::get + ); + Assert.assertEquals("This exception simulates an error", thrown.getCause().getMessage()); + GuavaUtils.cancelAll(true, future, futures); + Assert.assertEquals(0, futures.stream().filter(f -> !f.isDone()).count()); + for (CountDownLatch latch : latches) { + latch.countDown(); } } } From 4bd6bd0d4f61effa392cb5736d392ca12f7f3b40 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 27 Jun 2023 13:23:29 +0530 Subject: [PATCH 37/74] Improve CostBalancerStrategy, deprecate cachingCost (#14484) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Changes to `cost` strategy: - In every `ServerHolder`, track the number of segments per datasource per interval - Perform cost computations for a given interval just once, and then multiply by a constant factor to account for the total segment count in that interval - Do not perform joint cost computations with segments that are outside the compute interval (± 45 days) for the segment being considered for move - Remove metrics `segment/cost/*` as they were coordinator killers! Turning on these metrics (by setting `emitBalancingStats` to true) has often caused the coordinator to be stuck for hours. Moreover, they are too complicated to decipher and do not provide any meaningful insight into a Druid cluster. - Add new simpler metrics `segment/balancer/compute/*` to track cost computation time, count and errors. Other changes: - Remove flaky test from `CostBalancerStrategyTest`. - Add tests to verify that computed cost has remained unchanged - Remove usages of mock `BalancerStrategy` from `LoadRuleTest`, `BalanceSegmentsTest` - Clean up `BalancerStrategy` interface --- .../server/coordinator/DruidCoordinator.java | 24 +- .../coordinator/SegmentCountsPerInterval.java | 67 +++++ .../server/coordinator/ServerHolder.java | 68 ++--- .../balancer/BalancerStrategy.java | 56 ++-- .../balancer/CachingCostBalancerStrategy.java | 9 +- .../CachingCostBalancerStrategyFactory.java | 4 + .../balancer/CostBalancerStrategy.java | 268 +++++++++--------- .../DiskNormalizedCostBalancerStrategy.java | 7 +- .../balancer/RandomBalancerStrategy.java | 14 +- .../balancer/TierSegmentBalancer.java | 19 +- .../coordinator/duty/BalanceSegments.java | 6 + .../duty/CollectSegmentAndServerStats.java | 26 +- .../coordinator/duty/CompactSegments.java | 3 +- .../MarkOvershadowedSegmentsAsUnused.java | 5 +- .../server/coordinator/duty/RunRules.java | 2 +- .../loading/HttpLoadQueuePeon.java | 6 +- .../loading/SegmentLoadingConfig.java | 2 +- .../loading/StrategicSegmentAssigner.java | 26 +- .../stats/CoordinatorRunStats.java | 21 +- .../server/coordinator/stats/RowKey.java | 23 +- .../druid/server/coordinator/stats/Stats.java | 12 +- .../coordinator/CoordinatorRunStatsTest.java | 16 +- .../CachingCostBalancerStrategyTest.java | 3 +- .../balancer/CostBalancerStrategyTest.java | 137 +++++---- .../coordinator/duty/BalanceSegmentsTest.java | 81 ++---- .../coordinator/duty/CompactSegmentsTest.java | 2 +- .../server/coordinator/duty/RunRulesTest.java | 13 +- .../coordinator/rules/LoadRuleTest.java | 158 +++-------- 28 files changed, 513 insertions(+), 565 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/SegmentCountsPerInterval.java diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 0c5e79bdcbe..c9d334968ef 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; @@ -98,6 +99,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -672,7 +674,7 @@ public class DruidCoordinator { try { log.info("Starting coordinator run for group [%s]", dutyGroupName); - final long globalStart = System.currentTimeMillis(); + final Stopwatch groupRunTime = Stopwatch.createStarted(); synchronized (lock) { if (!coordLeaderSelector.isLeader()) { @@ -719,23 +721,25 @@ public class DruidCoordinator log.info("Coordination has been paused. Duties will not run until coordination is resumed."); } + final Stopwatch dutyRunTime = Stopwatch.createUnstarted(); for (CoordinatorDuty duty : duties) { // Don't read state and run state in the same duty otherwise racy conditions may exist if (!coordinationPaused && coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) { - final long start = System.currentTimeMillis(); + dutyRunTime.reset().start(); params = duty.run(params); - final long end = System.currentTimeMillis(); + dutyRunTime.stop(); final String dutyName = duty.getClass().getName(); if (params == null) { log.info("Stopping run for group [%s] on request of duty [%s].", dutyGroupName, dutyName); return; } else { - final RowKey rowKey = RowKey.builder().add(Dimension.DUTY, dutyName).build(); - params.getCoordinatorStats().add(Stats.CoordinatorRun.DUTY_RUN_TIME, rowKey, end - start); + final RowKey rowKey = RowKey.of(Dimension.DUTY, dutyName); + final long dutyRunMillis = dutyRunTime.elapsed(TimeUnit.MILLISECONDS); + params.getCoordinatorStats().add(Stats.CoordinatorRun.DUTY_RUN_TIME, rowKey, dutyRunMillis); } } } @@ -745,9 +749,9 @@ public class DruidCoordinator if (allStats.rowCount() > 0) { final AtomicInteger emittedCount = new AtomicInteger(); allStats.forEachStat( - (dimensionValues, stat, value) -> { + (stat, dimensions, value) -> { if (stat.shouldEmit()) { - emitStat(stat, dimensionValues, value); + emitStat(stat, dimensions.getValues(), value); emittedCount.incrementAndGet(); } } @@ -760,7 +764,7 @@ public class DruidCoordinator } // Emit the runtime of the full DutiesRunnable - final long runMillis = System.currentTimeMillis() - globalStart; + final long runMillis = groupRunTime.stop().elapsed(TimeUnit.MILLISECONDS); emitStat(Stats.CoordinatorRun.GROUP_RUN_TIME, Collections.emptyMap(), runMillis); log.info("Finished coordinator run for group [%s] in [%d] ms", dutyGroupName, runMillis); } @@ -771,10 +775,6 @@ public class DruidCoordinator private void emitStat(CoordinatorStat stat, Map dimensionValues, long value) { - if (stat.equals(Stats.Balancer.NORMALIZED_COST_X_1000)) { - value = value / 1000; - } - ServiceMetricEvent.Builder eventBuilder = new ServiceMetricEvent.Builder() .setDimension(Dimension.DUTY_GROUP.reportedName(), dutyGroupName); dimensionValues.forEach( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/SegmentCountsPerInterval.java b/server/src/main/java/org/apache/druid/server/coordinator/SegmentCountsPerInterval.java new file mode 100644 index 00000000000..ea1f81ee6d4 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/SegmentCountsPerInterval.java @@ -0,0 +1,67 @@ +/* + * 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.druid.server.coordinator; + +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntMaps; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.util.HashMap; +import java.util.Map; + +/** + * Maintains a count of segments for each datasource and interval. + */ +public class SegmentCountsPerInterval +{ + private final Map> + datasourceIntervalToSegmentCount = new HashMap<>(); + private final Object2IntMap intervalToTotalSegmentCount = new Object2IntOpenHashMap<>(); + + public void addSegment(DataSegment segment) + { + updateCountInInterval(segment, 1); + } + + public void removeSegment(DataSegment segment) + { + updateCountInInterval(segment, -1); + } + + public Object2IntMap getIntervalToSegmentCount(String datasource) + { + return datasourceIntervalToSegmentCount.getOrDefault(datasource, Object2IntMaps.emptyMap()); + } + + public Object2IntMap getIntervalToTotalSegmentCount() + { + return intervalToTotalSegmentCount; + } + + private void updateCountInInterval(DataSegment segment, int delta) + { + intervalToTotalSegmentCount.mergeInt(segment.getInterval(), delta, Integer::sum); + datasourceIntervalToSegmentCount + .computeIfAbsent(segment.getDataSource(), ds -> new Object2IntOpenHashMap<>()) + .mergeInt(segment.getInterval(), delta, Integer::sum); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java index 79f2c38ab20..7947f1c1b32 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java @@ -32,11 +32,9 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; /** @@ -75,11 +73,7 @@ public class ServerHolder implements Comparable */ private final Map queuedSegments = new HashMap<>(); - /** - * Segments that are expected to be loaded on this server once all the - * operations in progress have completed. - */ - private final Set projectedSegments = new HashSet<>(); + private final SegmentCountsPerInterval projectedSegments = new SegmentCountsPerInterval(); public ServerHolder(ImmutableDruidServer server, LoadQueuePeon peon) { @@ -133,31 +127,31 @@ public class ServerHolder implements Comparable AtomicInteger loadingReplicaCount ) { - projectedSegments.addAll(server.iterateAllSegments()); + for (DataSegment segment : server.iterateAllSegments()) { + projectedSegments.addSegment(segment); + } final List expiredSegments = new ArrayList<>(); - peon.getSegmentsInQueue().forEach( - (holder) -> { - int runsInQueue = holder.incrementAndGetRunsInQueue(); - if (runsInQueue > maxLifetimeInQueue) { - expiredSegments.add(holder); - } + for (SegmentHolder holder : peon.getSegmentsInQueue()) { + int runsInQueue = holder.incrementAndGetRunsInQueue(); + if (runsInQueue > maxLifetimeInQueue) { + expiredSegments.add(holder); + } - final SegmentAction action = holder.getAction(); - addToQueuedSegments(holder.getSegment(), simplify(action)); + final SegmentAction action = holder.getAction(); + addToQueuedSegments(holder.getSegment(), simplify(action)); - if (action == SegmentAction.MOVE_TO) { - movingSegmentCount.incrementAndGet(); - } - if (action == SegmentAction.REPLICATE) { - loadingReplicaCount.incrementAndGet(); - } - } - ); + if (action == SegmentAction.MOVE_TO) { + movingSegmentCount.incrementAndGet(); + } + if (action == SegmentAction.REPLICATE) { + loadingReplicaCount.incrementAndGet(); + } + } - peon.getSegmentsMarkedToDrop().forEach( - segment -> addToQueuedSegments(segment, SegmentAction.MOVE_FROM) - ); + for (DataSegment segment : peon.getSegmentsMarkedToDrop()) { + addToQueuedSegments(segment, SegmentAction.MOVE_FROM); + } if (!expiredSegments.isEmpty()) { List expiredSegmentsSubList = @@ -251,11 +245,21 @@ public class ServerHolder implements Comparable * Segments that are expected to be loaded on this server once all the * operations in progress have completed. */ - public Set getProjectedSegments() + public SegmentCountsPerInterval getProjectedSegments() { return projectedSegments; } + public boolean isProjectedSegment(DataSegment segment) + { + SegmentAction action = getActionOnSegment(segment); + if (action == null) { + return hasSegmentLoaded(segment.getId()); + } else { + return action.isLoad(); + } + } + /** * Segments that are currently in the queue for being loaded on this server. * This does not include segments that are being moved to this server. @@ -362,10 +366,10 @@ public class ServerHolder implements Comparable // Add to projected if load is started, remove from projected if drop has started if (action.isLoad()) { - projectedSegments.add(segment); + projectedSegments.addSegment(segment); sizeOfLoadingSegments += segment.getSize(); } else { - projectedSegments.remove(segment); + projectedSegments.removeSegment(segment); if (action == SegmentAction.DROP) { sizeOfDroppingSegments += segment.getSize(); } @@ -379,10 +383,10 @@ public class ServerHolder implements Comparable queuedSegments.remove(segment); if (action.isLoad()) { - projectedSegments.remove(segment); + projectedSegments.removeSegment(segment); sizeOfLoadingSegments -= segment.getSize(); } else { - projectedSegments.add(segment); + projectedSegments.addSegment(segment); if (action == SegmentAction.DROP) { sizeOfDroppingSegments -= segment.getSize(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java index 7f27648d3f8..e133430b1b2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java @@ -20,70 +20,60 @@ package org.apache.druid.server.coordinator.balancer; import org.apache.druid.server.coordinator.ServerHolder; -import org.apache.druid.server.coordinator.duty.BalanceSegments; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.timeline.DataSegment; import javax.annotation.Nullable; import java.util.Iterator; import java.util.List; -import java.util.NavigableSet; /** - * This interface describes the coordinator balancing strategy, which is responsible for making decisions on where - * to place {@link DataSegment}s on historical servers (described by {@link ServerHolder}). The balancing strategy - * is used by {@link org.apache.druid.server.coordinator.rules.LoadRule} to assign and drop segments, and by - * {@link BalanceSegments} to migrate segments between historicals. + * Segment balancing strategy, used in every coordinator run by + * {@link org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner} + * to choose optimal servers to load, move or drop a segment. */ public interface BalancerStrategy { /** - * Finds the best server to move a segment to according to the balancing strategy. + * Finds the best server from the list of {@code destinationServers} to load + * the {@code segmentToMove}, if it is moved from the {@code sourceServer}. + *

+ * In order to avoid unnecessary moves when the segment is already optimally placed, + * include the {@code sourceServer} in the list of {@code destinationServers}. * - * @param proposalSegment segment to move - * @param sourceServer Server the segment is currently placed on. - * @param destinationServers servers to consider as move destinations - * @return The server to move to, or null if no move should be made or no server is suitable + * @return The server to move to, or null if the segment is already optimally placed. */ @Nullable ServerHolder findDestinationServerToMoveSegment( - DataSegment proposalSegment, + DataSegment segmentToMove, ServerHolder sourceServer, List destinationServers ); /** - * Finds the best servers on which to place the {@code proposalSegment}. - * This method can be used both for placing the first copy of a segment - * in the tier or a replica of the segment. + * Finds the best servers to load the given segment. This method can be used + * both for placing the first copy of a segment in a tier or a replica of an + * already available segment. * - * @param proposalSegment segment to place on servers - * @param serverHolders servers to consider as segment homes - * @return Iterator over the best servers (in order) on which the segment - * can be placed. + * @return Iterator over the best servers (in order of preference) to load + * the segment. */ Iterator findServersToLoadSegment( - DataSegment proposalSegment, + DataSegment segmentToLoad, List serverHolders ); /** - * Returns an iterator for a set of servers to drop from, ordered by preference of which server to drop from first - * for a given drop strategy. One or more segments may be dropped, depending on how much the segment is - * over-replicated. - * @param toDropSegment segment to drop from one or more servers - * @param serverHolders set of historicals to consider dropping from - * @return Iterator for set of historicals, ordered by drop preference + * Finds the best servers to drop the given segment. + * + * @return Iterator over the servers (in order of preference) to drop the segment */ - Iterator pickServersToDropSegment(DataSegment toDropSegment, NavigableSet serverHolders); + Iterator findServersToDropSegment(DataSegment segmentToDrop, List serverHolders); /** - * Add balancing strategy stats during the 'balanceTier' operation of - * {@link BalanceSegments} to be included - * @param tier historical tier being balanced - * @param stats stats object to add balancing strategy stats to - * @param serverHolderList servers in tier being balanced + * Returns the stats collected by the strategy in the current run and resets + * the stats collector for the next run. */ - void emitStats(String tier, CoordinatorRunStats stats, List serverHolderList); + CoordinatorRunStats getAndResetStats(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategy.java index 424657991cf..eda99289157 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategy.java @@ -28,10 +28,8 @@ import org.apache.druid.timeline.DataSegment; import java.util.Collections; import java.util.Set; - public class CachingCostBalancerStrategy extends CostBalancerStrategy { - private final ClusterCostCache clusterCostCache; public CachingCostBalancerStrategy(ClusterCostCache clusterCostCache, ListeningExecutorService exec) @@ -41,13 +39,8 @@ public class CachingCostBalancerStrategy extends CostBalancerStrategy } @Override - protected double computeCost(DataSegment proposalSegment, ServerHolder server, boolean includeCurrentServer) + protected double computePlacementCost(DataSegment proposalSegment, ServerHolder server) { - // (optional) Don't include server if it cannot load the segment - if (!includeCurrentServer && !server.canLoadSegment(proposalSegment)) { - return Double.POSITIVE_INFINITY; - } - final String serverName = server.getServer().getName(); double cost = clusterCostCache.computeCost(serverName, proposalSegment); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java index 726dffe6904..0ddacaead70 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyFactory.java @@ -125,6 +125,10 @@ public class CachingCostBalancerStrategyFactory implements BalancerStrategyFacto @Override public BalancerStrategy createBalancerStrategy(final ListeningExecutorService exec) { + LOG.warn( + "'cachingCost' balancer strategy has been deprecated as it can lead to" + + " unbalanced clusters. Use 'cost' strategy instead." + ); if (!isInitialized() && config.isAwaitInitialization()) { try { final long startMillis = System.currentTimeMillis(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategy.java index 6d165c1f349..aae907bb599 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategy.java @@ -19,16 +19,21 @@ package org.apache.druid.server.coordinator.balancer; +import com.google.common.base.Stopwatch; import com.google.common.collect.Lists; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.commons.math3.util.FastMath; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordinator.SegmentCountsPerInterval; import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.loading.SegmentAction; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; @@ -38,11 +43,10 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.Iterator; import java.util.List; -import java.util.NavigableSet; import java.util.PriorityQueue; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; public class CostBalancerStrategy implements BalancerStrategy { @@ -63,6 +67,20 @@ public class CostBalancerStrategy implements BalancerStrategy = Comparator., Double>comparing(pair -> pair.lhs) .thenComparing(pair -> pair.rhs); + private final CoordinatorRunStats stats = new CoordinatorRunStats(); + + public static double computeJointSegmentsCost(DataSegment segment, Iterable segmentSet) + { + final Interval costComputeInterval = getCostComputeInterval(segment); + double totalCost = 0; + for (DataSegment s : segmentSet) { + if (costComputeInterval.overlaps(s.getInterval())) { + totalCost += computeJointSegmentsCost(segment, s); + } + } + return totalCost; + } + /** * This defines the unnormalized cost function between two segments. * @@ -83,15 +101,20 @@ public class CostBalancerStrategy implements BalancerStrategy final Interval intervalA = segmentA.getInterval(); final Interval intervalB = segmentB.getInterval(); + // constant cost-multiplier for segments of the same datsource + final double multiplier = segmentA.getDataSource().equals(segmentB.getDataSource()) + ? 2.0 : 1.0; + return intervalCost(intervalA, intervalB) * multiplier; + } + + public static double intervalCost(Interval intervalA, Interval intervalB) + { final double t0 = intervalA.getStartMillis(); final double t1 = (intervalA.getEndMillis() - t0) / MILLIS_FACTOR; final double start = (intervalB.getStartMillis() - t0) / MILLIS_FACTOR; final double end = (intervalB.getEndMillis() - t0) / MILLIS_FACTOR; - // constant cost-multiplier for segments of the same datsource - final double multiplier = segmentA.getDataSource().equals(segmentB.getDataSource()) ? 2.0 : 1.0; - - return INV_LAMBDA_SQUARE * intervalCost(t1, start, end) * multiplier; + return INV_LAMBDA_SQUARE * intervalCost(t1, start, end); } /** @@ -199,161 +222,119 @@ public class CostBalancerStrategy implements BalancerStrategy @Override public Iterator findServersToLoadSegment( - DataSegment proposalSegment, + DataSegment segmentToLoad, List serverHolders ) { - return getServersByPlacementCost(proposalSegment, serverHolders, false, "findServersToLoadSegment"); + return orderServersByPlacementCost(segmentToLoad, serverHolders, SegmentAction.LOAD) + .stream() + .filter(server -> server.canLoadSegment(segmentToLoad)) + .iterator(); } - @Override public ServerHolder findDestinationServerToMoveSegment( - DataSegment proposalSegment, + DataSegment segmentToMove, ServerHolder sourceServer, List serverHolders ) { - Iterator servers = - getServersByPlacementCost(proposalSegment, serverHolders, true, "findServerToMoveSegment"); - return servers.hasNext() ? servers.next() : null; - } - - public static double computeJointSegmentsCost(DataSegment segment, Iterable segmentSet) - { - double totalCost = 0; - for (DataSegment s : segmentSet) { - totalCost += computeJointSegmentsCost(segment, s); + List servers = + orderServersByPlacementCost(segmentToMove, serverHolders, SegmentAction.MOVE_TO); + if (servers.isEmpty()) { + return null; } - return totalCost; + + ServerHolder candidateServer = servers.get(0); + return candidateServer.equals(sourceServer) ? null : candidateServer; } @Override - public Iterator pickServersToDropSegment( + public Iterator findServersToDropSegment( DataSegment segmentToDrop, - NavigableSet serverHolders + List serverHolders ) { - List serversByCost = Lists.newArrayList( - getServersByPlacementCost(segmentToDrop, serverHolders, true, "pickServersToDropSegment") - ); + List serversByCost = + orderServersByPlacementCost(segmentToDrop, serverHolders, SegmentAction.DROP); // Prioritize drop from highest cost servers return Lists.reverse(serversByCost).iterator(); } - /** - * Calculates the initial cost of the Druid segment configuration. - * - * @param serverHolders A list of ServerHolders for a particular tier. - * - * @return The initial cost of the Druid tier. - */ - public double calculateInitialTotalCost(final List serverHolders) - { - double cost = 0; - for (ServerHolder server : serverHolders) { - // segments are dumped into an array because it's probably better than iterating the iterateAllSegments() result - // quadratically in a loop, which can generate garbage in the form of Stream, Spliterator, Iterator, etc. objects - // whose total memory volume exceeds the size of the DataSegment array. - DataSegment[] segments = server.getServer().iterateAllSegments().toArray(new DataSegment[0]); - for (DataSegment s1 : segments) { - for (DataSegment s2 : segments) { - cost += computeJointSegmentsCost(s1, s2); - } - } - } - return cost; - } - - /** - * Calculates the cost normalization. This is such that the normalized cost is lower bounded - * by 1 (e.g. when each segment gets its own historical node). - * - * @param serverHolders A list of ServerHolders for a particular tier. - * - * @return The normalization value (the sum of the diagonal entries in the - * pairwise cost matrix). This is the cost of a cluster if each - * segment were to get its own historical node. - */ - public double calculateNormalization(final List serverHolders) - { - double cost = 0; - for (ServerHolder server : serverHolders) { - for (DataSegment segment : server.getServedSegments()) { - cost += computeJointSegmentsCost(segment, segment); - } - } - return cost; - } - @Override - public void emitStats(String tier, CoordinatorRunStats stats, List serverHolderList) + public CoordinatorRunStats getAndResetStats() { - final double initialTotalCost = calculateInitialTotalCost(serverHolderList); - final double normalization = calculateNormalization(serverHolderList); - final double normalizedInitialCost = initialTotalCost / normalization; - - final RowKey rowKey = RowKey.forTier(tier); - stats.add(Stats.Balancer.RAW_COST, rowKey, (long) initialTotalCost); - stats.add(Stats.Balancer.NORMALIZATION_COST, rowKey, (long) normalization); - stats.add(Stats.Balancer.NORMALIZED_COST_X_1000, rowKey, (long) (normalizedInitialCost * 1000)); - - log.info( - "[%s]: Initial Total Cost: [%f], Normalization: [%f], Initial Normalized Cost: [%f]", - tier, initialTotalCost, normalization, normalizedInitialCost - ); + return stats.getSnapshotAndReset(); } - protected double computeCost( - final DataSegment proposalSegment, - final ServerHolder server, - final boolean includeCurrentServer - ) + /** + * Computes the cost of placing a segment on this server. + */ + protected double computePlacementCost(DataSegment proposalSegment, ServerHolder server) { - // (optional) Don't include server if it cannot load the segment - if (!includeCurrentServer && !server.canLoadSegment(proposalSegment)) { - return Double.POSITIVE_INFINITY; - } + final Interval costComputeInterval = getCostComputeInterval(proposalSegment); - // The contribution to the total cost of a given server by proposing to move the segment to that server is... - double cost = 0d; + // Compute number of segments in each interval + final Object2IntOpenHashMap intervalToSegmentCount = new Object2IntOpenHashMap<>(); - // the sum of the costs of segments expected to be on the server (loaded + loading - dropping) - Set projectedSegments = server.getProjectedSegments(); - cost += computeJointSegmentsCost(proposalSegment, projectedSegments); + final SegmentCountsPerInterval projectedSegments = server.getProjectedSegments(); + projectedSegments.getIntervalToTotalSegmentCount().object2IntEntrySet().forEach(entry -> { + final Interval interval = entry.getKey(); + if (costComputeInterval.overlaps(interval)) { + intervalToSegmentCount.addTo(interval, entry.getIntValue()); + } + }); - // minus the self cost of the segment - if (projectedSegments.contains(proposalSegment)) { - cost -= computeJointSegmentsCost(proposalSegment, proposalSegment); + // Count the segments for the same datasource twice as they have twice the cost + final String datasource = proposalSegment.getDataSource(); + projectedSegments.getIntervalToSegmentCount(datasource).object2IntEntrySet().forEach(entry -> { + final Interval interval = entry.getKey(); + if (costComputeInterval.overlaps(interval)) { + intervalToSegmentCount.addTo(interval, entry.getIntValue()); + } + }); + + // Compute joint cost for each interval + double cost = 0; + final Interval segmentInterval = proposalSegment.getInterval(); + cost += intervalToSegmentCount.object2IntEntrySet().stream().mapToDouble( + entry -> intervalCost(segmentInterval, entry.getKey()) + * entry.getIntValue() + ).sum(); + + // Minus the self cost of the segment + if (server.isProjectedSegment(proposalSegment)) { + cost -= intervalCost(segmentInterval, segmentInterval) * 2.0; } return cost; } /** - * Returns an iterator over the servers, ordered by increasing cost for - * placing the given segment on that server. - * - * @param includeCurrentServer true if the server already serving a replica - * of this segment should be included in the results + * Orders the servers by increasing cost for placing the given segment. */ - private Iterator getServersByPlacementCost( - DataSegment proposalSegment, - Iterable serverHolders, - boolean includeCurrentServer, - String action + private List orderServersByPlacementCost( + DataSegment segment, + List serverHolders, + SegmentAction action ) { + final Stopwatch computeTime = Stopwatch.createStarted(); final List>> futures = new ArrayList<>(); for (ServerHolder server : serverHolders) { futures.add( exec.submit( - () -> Pair.of(computeCost(proposalSegment, server, includeCurrentServer), server) + () -> Pair.of(computePlacementCost(segment, server), server) ) ); } + String tier = serverHolders.isEmpty() ? null : serverHolders.get(0).getServer().getTier(); + final RowKey metricKey = RowKey.with(Dimension.TIER, tier) + .with(Dimension.DATASOURCE, segment.getDataSource()) + .and(Dimension.DESCRIPTION, action.name()); + final PriorityQueue> costPrioritizedServers = new PriorityQueue<>(CHEAPEST_SERVERS_FIRST); try { @@ -364,30 +345,57 @@ public class CostBalancerStrategy implements BalancerStrategy ); } catch (Exception e) { - alertOnFailure(e, action); + stats.add(Stats.Balancer.COMPUTATION_ERRORS, metricKey, 1); + handleFailure(e, segment, action); } - // Include current server only if specified - return costPrioritizedServers.stream() - .filter(pair -> includeCurrentServer || pair.rhs.canLoadSegment(proposalSegment)) - .map(pair -> pair.rhs).iterator(); + // Report computation stats + computeTime.stop(); + stats.add(Stats.Balancer.COMPUTATION_COUNT, metricKey, 1); + stats.add(Stats.Balancer.COMPUTATION_TIME, metricKey, computeTime.elapsed(TimeUnit.MILLISECONDS)); + + return costPrioritizedServers.stream().map(pair -> pair.rhs) + .collect(Collectors.toList()); } - private void alertOnFailure(Exception e, String action) + private void handleFailure( + Exception e, + DataSegment segment, + SegmentAction action + ) { - // Do not alert if the executor has been shutdown + final String reason; + String suggestion = ""; if (exec.isShutdown()) { - log.noStackTrace().info("Balancer executor was terminated. Failing action [%s].", action); - return; + reason = "Executor shutdown"; + } else if (e instanceof TimeoutException) { + reason = "Timed out"; + suggestion = " Try setting a higher value for 'balancerComputeThreads'."; + } else { + reason = e.getMessage(); } - final boolean hasTimedOut = e instanceof TimeoutException; - final String message = StringUtils.format( - "Cost balancer strategy %s in action [%s].%s", - hasTimedOut ? "timed out" : "failed", action, - hasTimedOut ? " Try setting a higher value of 'balancerComputeThreads'." : "" - ); - log.makeAlert(e, message).emit(); + String msgFormat = "Cost strategy computations failed for action[%s] on segment[%s] due to reason[%s].[%s]"; + log.noStackTrace().warn(e, msgFormat, action, segment.getId(), reason, suggestion); + } + + /** + * The cost compute interval for a segment is {@code [start-45days, end+45days)}. + * This is because the joint cost of any two segments that are 45 days apart is + * negligible. + */ + private static Interval getCostComputeInterval(DataSegment segment) + { + final Interval segmentInterval = segment.getInterval(); + if (Intervals.isEternity(segmentInterval)) { + return segmentInterval; + } else { + final long maxGap = TimeUnit.DAYS.toMillis(45); + return Intervals.utc( + segmentInterval.getStartMillis() - maxGap, + segmentInterval.getEndMillis() + maxGap + ); + } } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategy.java index cee292930cf..601e5b042e0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/DiskNormalizedCostBalancerStrategy.java @@ -35,13 +35,12 @@ public class DiskNormalizedCostBalancerStrategy extends CostBalancerStrategy * This ensures that all the hosts will have the same % disk utilization. */ @Override - protected double computeCost( + protected double computePlacementCost( final DataSegment proposalSegment, - final ServerHolder server, - final boolean includeCurrentServer + final ServerHolder server ) { - double cost = super.computeCost(proposalSegment, server, includeCurrentServer); + double cost = super.computePlacementCost(proposalSegment, server); if (cost == Double.POSITIVE_INFINITY) { return cost; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategy.java index 98b1d8bdc70..cccc2518e83 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/RandomBalancerStrategy.java @@ -27,7 +27,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.NavigableSet; import java.util.stream.Collectors; /** @@ -39,16 +38,18 @@ import java.util.stream.Collectors; */ public class RandomBalancerStrategy implements BalancerStrategy { + private static final CoordinatorRunStats EMPTY_STATS = new CoordinatorRunStats(); + @Override public Iterator findServersToLoadSegment( - DataSegment proposalSegment, + DataSegment segmentToLoad, List serverHolders ) { // Filter out servers which cannot load this segment final List usableServerHolders = serverHolders.stream() - .filter(server -> server.canLoadSegment(proposalSegment)) + .filter(server -> server.canLoadSegment(segmentToLoad)) .collect(Collectors.toList()); Collections.shuffle(usableServerHolders); return usableServerHolders.iterator(); @@ -56,7 +57,7 @@ public class RandomBalancerStrategy implements BalancerStrategy @Override public ServerHolder findDestinationServerToMoveSegment( - DataSegment proposalSegment, + DataSegment segmentToMove, ServerHolder sourceServer, List serverHolders ) @@ -66,7 +67,7 @@ public class RandomBalancerStrategy implements BalancerStrategy } @Override - public Iterator pickServersToDropSegment(DataSegment toDropSegment, NavigableSet serverHolders) + public Iterator findServersToDropSegment(DataSegment segmentToDrop, List serverHolders) { List serverList = new ArrayList<>(serverHolders); Collections.shuffle(serverList); @@ -74,7 +75,8 @@ public class RandomBalancerStrategy implements BalancerStrategy } @Override - public void emitStats(String tier, CoordinatorRunStats stats, List serverHolderList) + public CoordinatorRunStats getAndResetStats() { + return EMPTY_STATS; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java index a5ab8d0d477..86e5b996293 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordinator.balancer; -import com.google.common.collect.Lists; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -56,11 +55,9 @@ public class TierSegmentBalancer private final DruidCoordinatorRuntimeParams params; private final StrategicSegmentAssigner segmentAssigner; - private final BalancerStrategy strategy; private final SegmentLoadingConfig loadingConfig; private final CoordinatorRunStats runStats; - private final Set allServers; private final List activeServers; private final List decommissioningServers; private final int totalMaxSegmentsToMove; @@ -77,7 +74,6 @@ public class TierSegmentBalancer this.params = params; this.segmentAssigner = params.getSegmentAssigner(); - this.strategy = params.getBalancerStrategy(); this.loadingConfig = params.getSegmentLoadingConfig(); this.totalMaxSegmentsToMove = loadingConfig.getMaxSegmentsToMove(); this.runStats = segmentAssigner.getStats(); @@ -86,7 +82,6 @@ public class TierSegmentBalancer servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning)); this.decommissioningServers = partitions.get(true); this.activeServers = partitions.get(false); - this.allServers = servers; this.movingSegmentCount = activeServers.stream().mapToInt(ServerHolder::getNumMovingSegments).sum(); } @@ -128,10 +123,6 @@ public class TierSegmentBalancer "Moved [%d] segments out of max [%d] between active servers in tier [%s].", movedGeneralSegments, maxGeneralSegmentsToMove, tier ); - - if (loadingConfig.isEmitBalancingStats()) { - strategy.emitStats(tier, runStats, Lists.newArrayList(allServers)); - } } private int moveSegmentsFromTo( @@ -224,13 +215,9 @@ public class TierSegmentBalancer private void markUnmoved(String reason, DataSegment segment) { - final RowKey key - = RowKey.builder() - .add(Dimension.TIER, tier) - .add(Dimension.DATASOURCE, segment.getDataSource()) - .add(Dimension.DESCRIPTION, reason) - .build(); - + RowKey key = RowKey.with(Dimension.TIER, tier) + .with(Dimension.DATASOURCE, segment.getDataSource()) + .and(Dimension.DESCRIPTION, reason); runStats.add(Stats.Segments.MOVE_SKIPPED, key, 1); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java index 865fde2a0bc..4ff2e657438 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java @@ -24,6 +24,7 @@ import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.balancer.TierSegmentBalancer; import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; /** * @@ -57,6 +58,11 @@ public class BalanceSegments implements CoordinatorDuty (tier, servers) -> new TierSegmentBalancer(tier, servers, params).run() ); + CoordinatorRunStats runStats = params.getCoordinatorStats(); + params.getBalancerStrategy() + .getAndResetStats() + .forEachStat(runStats::add); + return params; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java index b5825cb084f..5109ec3c3f2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CollectSegmentAndServerStats.java @@ -72,7 +72,7 @@ public class CollectSegmentAndServerStats implements CoordinatorDuty final DruidCluster cluster = params.getDruidCluster(); cluster.getHistoricals().forEach((tier, historicals) -> { - final RowKey rowKey = RowKey.forTier(tier); + final RowKey rowKey = RowKey.of(Dimension.TIER, tier); stats.add(Stats.Tier.HISTORICAL_COUNT, rowKey, historicals.size()); long totalCapacity = historicals.stream().map(ServerHolder::getMaxSize).reduce(0L, Long::sum); stats.add(Stats.Tier.TOTAL_CAPACITY, rowKey, totalCapacity); @@ -80,20 +80,23 @@ public class CollectSegmentAndServerStats implements CoordinatorDuty // Collect load queue stats coordinator.getLoadManagementPeons().forEach((serverName, queuePeon) -> { - final RowKey rowKey = RowKey.builder().add(Dimension.SERVER, serverName).build(); + final RowKey rowKey = RowKey.of(Dimension.SERVER, serverName); stats.add(Stats.SegmentQueue.BYTES_TO_LOAD, rowKey, queuePeon.getSizeOfSegmentsToLoad()); stats.add(Stats.SegmentQueue.NUM_TO_LOAD, rowKey, queuePeon.getSegmentsToLoad().size()); stats.add(Stats.SegmentQueue.NUM_TO_DROP, rowKey, queuePeon.getSegmentsToDrop().size()); queuePeon.getAndResetStats().forEachStat( - (dimValues, stat, statValue) -> - stats.add(stat, createRowKeyForServer(serverName, dimValues), statValue) + (stat, key, statValue) -> + stats.add(stat, createRowKeyForServer(serverName, key.getValues()), statValue) ); }); coordinator.getDatasourceToUnavailableSegmentCount().forEach( - (dataSource, numUnavailable) -> - stats.addToDatasourceStat(Stats.Segments.UNAVAILABLE, dataSource, numUnavailable) + (dataSource, numUnavailable) -> stats.add( + Stats.Segments.UNAVAILABLE, + RowKey.of(Dimension.DATASOURCE, dataSource), + numUnavailable + ) ); coordinator.getTierToDatasourceToUnderReplicatedCount(false).forEach( @@ -108,17 +111,18 @@ public class CollectSegmentAndServerStats implements CoordinatorDuty (dataSource, timeline) -> { long totalSizeOfUsedSegments = timeline.iterateAllObjects().stream() .mapToLong(DataSegment::getSize).sum(); - stats.addToDatasourceStat(Stats.Segments.USED_BYTES, dataSource, totalSizeOfUsedSegments); - stats.addToDatasourceStat(Stats.Segments.USED, dataSource, timeline.getNumObjects()); + + RowKey datasourceKey = RowKey.of(Dimension.DATASOURCE, dataSource); + stats.add(Stats.Segments.USED_BYTES, datasourceKey, totalSizeOfUsedSegments); + stats.add(Stats.Segments.USED, datasourceKey, timeline.getNumObjects()); } ); } private RowKey createRowKeyForServer(String serverName, Map dimensionValues) { - final RowKey.Builder builder = RowKey.builder(); - dimensionValues.forEach(builder::add); - builder.add(Dimension.SERVER, serverName); + final RowKey.Builder builder = RowKey.with(Dimension.SERVER, serverName); + dimensionValues.forEach(builder::with); return builder.build(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index d767c51a925..0381a70ba24 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -43,6 +43,7 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; @@ -593,7 +594,7 @@ public class CompactSegments implements CoordinatorCustomDuty CoordinatorRunStats stats ) { - final RowKey rowKey = RowKey.forDatasource(dataSource); + final RowKey rowKey = RowKey.of(Dimension.DATASOURCE, dataSource); stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction()); stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction()); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnused.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnused.java index 1a63083bf77..3cb9f0064cf 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnused.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnused.java @@ -28,6 +28,8 @@ import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -100,7 +102,8 @@ public class MarkOvershadowedSegmentsAsUnused implements CoordinatorDuty final CoordinatorRunStats stats = params.getCoordinatorStats(); datasourceToUnusedSegments.forEach( (datasource, unusedSegments) -> { - stats.addToDatasourceStat(Stats.Segments.OVERSHADOWED, datasource, unusedSegments.size()); + RowKey datasourceKey = RowKey.of(Dimension.DATASOURCE, datasource); + stats.add(Stats.Segments.OVERSHADOWED, datasourceKey, unusedSegments.size()); coordinator.markSegmentsAsUnused(datasource, unusedSegments); } ); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java index eb037d1212f..a5c4eb58041 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java @@ -67,7 +67,7 @@ public class RunRules implements CoordinatorDuty final Set overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments(); final Set usedSegments = params.getUsedSegments(); log.info( - "Applying retention rules on [%d] used segments, skipping [%d] overshadowed segments.", + "Applying retention rules on [%,d] used segments, skipping [%,d] overshadowed segments.", usedSegments.size(), overshadowed.size() ); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 9c133fce8c6..92dd4714a45 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -40,6 +40,8 @@ import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; @@ -543,7 +545,9 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private void incrementStat(SegmentHolder holder, RequestStatus status) { - stats.addToDatasourceStat(status.datasourceStat, holder.getSegment().getDataSource(), 1); + RowKey rowKey = RowKey.with(Dimension.DATASOURCE, holder.getSegment().getDataSource()) + .and(Dimension.DESCRIPTION, holder.getAction().name()); + stats.add(status.datasourceStat, rowKey, 1); } private void executeCallbacks(SegmentHolder holder, boolean success) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java index 559950c18e7..fbf867fd91f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java @@ -61,7 +61,7 @@ public class SegmentLoadingConfig log.info( "Smart segment loading is enabled. Recomputed replicationThrottleLimit" - + " [%,d] (%d%% of used segments [%,d]) and maxSegmentsToMove [%d].", + + " [%,d] (%d%% of used segments [%,d]) and maxSegmentsToMove [%,d].", replicationThrottleLimit, throttlePercentage, numUsedSegments, maxSegmentsToMove ); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java index 4fc2e176f75..6546cce69c7 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java @@ -33,6 +33,7 @@ import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; +import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -296,7 +297,7 @@ public class StrategicSegmentAssigner implements SegmentActionHandler private void reportTierCapacityStats(DataSegment segment, int requiredReplicas, String tier) { - final RowKey rowKey = RowKey.forTier(tier); + final RowKey rowKey = RowKey.of(Dimension.TIER, tier); stats.updateMax(Stats.Tier.REPLICATION_FACTOR, rowKey, requiredReplicas); stats.add(Stats.Tier.REQUIRED_CAPACITY, rowKey, segment.getSize() * requiredReplicas); } @@ -342,7 +343,8 @@ public class StrategicSegmentAssigner implements SegmentActionHandler public void deleteSegment(DataSegment segment) { loadQueueManager.deleteSegment(segment); - stats.addToDatasourceStat(Stats.Segments.DELETED, segment.getDataSource(), 1); + RowKey rowKey = RowKey.of(Dimension.DATASOURCE, segment.getDataSource()); + stats.add(Stats.Segments.DELETED, rowKey, 1); } /** @@ -429,9 +431,9 @@ public class StrategicSegmentAssigner implements SegmentActionHandler if (numToDrop > numDropsQueued) { remainingNumToDrop = numToDrop - numDropsQueued; Iterator serverIterator = - (useRoundRobinAssignment || eligibleLiveServers.size() >= remainingNumToDrop) + (useRoundRobinAssignment || eligibleLiveServers.size() <= remainingNumToDrop) ? eligibleLiveServers.iterator() - : strategy.pickServersToDropSegment(segment, eligibleLiveServers); + : strategy.findServersToDropSegment(segment, new ArrayList<>(eligibleLiveServers)); numDropsQueued += dropReplicasFromServers(remainingNumToDrop, segment, serverIterator, tier); } @@ -493,7 +495,7 @@ public class StrategicSegmentAssigner implements SegmentActionHandler ? serverSelector.getServersInTierToLoadSegment(tier, segment) : strategy.findServersToLoadSegment(segment, eligibleServers); if (!serverIterator.hasNext()) { - incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No server chosen by strategy", segment, tier); + incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No strategic server", segment, tier); return 0; } @@ -586,16 +588,10 @@ public class StrategicSegmentAssigner implements SegmentActionHandler private void incrementSkipStat(CoordinatorStat stat, String reason, DataSegment segment, String tier) { - final RowKey.Builder keyBuilder - = RowKey.builder() - .add(Dimension.TIER, tier) - .add(Dimension.DATASOURCE, segment.getDataSource()); - - if (reason != null) { - keyBuilder.add(Dimension.DESCRIPTION, reason); - } - - stats.add(stat, keyBuilder.build(), 1); + final RowKey key = RowKey.with(Dimension.TIER, tier) + .with(Dimension.DATASOURCE, segment.getDataSource()) + .and(Dimension.DESCRIPTION, reason); + stats.add(stat, key, 1); } private void incrementStat(CoordinatorStat stat, DataSegment segment, String tier, long value) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java index 7d63c78c479..f8ea6cf98f8 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java @@ -64,12 +64,7 @@ public class CoordinatorRunStats public long getSegmentStat(CoordinatorStat stat, String tier, String datasource) { - return get(stat, RowKey.builder().add(Dimension.DATASOURCE, datasource).add(Dimension.TIER, tier).build()); - } - - public long getDataSourceStat(CoordinatorStat stat, String dataSource) - { - return get(stat, RowKey.forDatasource(dataSource)); + return get(stat, RowKey.with(Dimension.DATASOURCE, datasource).and(Dimension.TIER, tier)); } public long get(CoordinatorStat stat) @@ -87,7 +82,7 @@ public class CoordinatorRunStats { allStats.forEach( (rowKey, stats) -> stats.object2LongEntrySet().fastForEach( - stat -> handler.handle(rowKey.getValues(), stat.getKey(), stat.getLongValue()) + stat -> handler.handle(stat.getKey(), rowKey, stat.getLongValue()) ) ); } @@ -199,16 +194,10 @@ public class CoordinatorRunStats .addTo(stat, value); } - public void addToDatasourceStat(CoordinatorStat stat, String dataSource, long value) - { - add(stat, RowKey.forDatasource(dataSource), value); - } - public void addToSegmentStat(CoordinatorStat stat, String tier, String datasource, long value) { - RowKey rowKey = RowKey.builder() - .add(Dimension.TIER, tier) - .add(Dimension.DATASOURCE, datasource).build(); + RowKey rowKey = RowKey.with(Dimension.TIER, tier) + .and(Dimension.DATASOURCE, datasource); add(stat, rowKey, value); } @@ -267,7 +256,7 @@ public class CoordinatorRunStats public interface StatHandler { - void handle(Map dimensionValues, CoordinatorStat stat, long statValue); + void handle(CoordinatorStat stat, RowKey rowKey, long statValue); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java index 1893f86a9cd..b0ee0a2d1f7 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/RowKey.java @@ -40,19 +40,16 @@ public class RowKey this.hashCode = Objects.hash(values); } - public static Builder builder() + public static Builder with(Dimension dimension, String value) { - return new RowKey.Builder(); + Builder builder = new Builder(); + builder.with(dimension, value); + return builder; } - public static RowKey forTier(String tier) + public static RowKey of(Dimension dimension, String value) { - return RowKey.builder().add(Dimension.TIER, tier).build(); - } - - public static RowKey forDatasource(String datasource) - { - return RowKey.builder().add(Dimension.DATASOURCE, datasource).build(); + return with(dimension, value).build(); } public Map getValues() @@ -83,12 +80,18 @@ public class RowKey { private final Map values = new EnumMap<>(Dimension.class); - public Builder add(Dimension dimension, String value) + public Builder with(Dimension dimension, String value) { values.put(dimension, value); return this; } + public RowKey and(Dimension dimension, String value) + { + values.put(dimension, value); + return new RowKey(values); + } + public RowKey build() { return new RowKey(values); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java index 791d3963dd8..9864aa6b3ab 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java @@ -131,11 +131,11 @@ public class Stats public static class Balancer { - public static final CoordinatorStat RAW_COST - = new CoordinatorStat("initialCost", "segment/cost/raw"); - public static final CoordinatorStat NORMALIZATION_COST - = new CoordinatorStat("normaliznCost", "segment/cost/normalization"); - public static final CoordinatorStat NORMALIZED_COST_X_1000 - = new CoordinatorStat("normalizedCost", "segment/cost/normalized"); + public static final CoordinatorStat COMPUTATION_ERRORS + = new CoordinatorStat("costComputeError", "segment/balancer/compute/error"); + public static final CoordinatorStat COMPUTATION_TIME + = new CoordinatorStat("costComputeTime", "segment/balancer/compute/time"); + public static final CoordinatorStat COMPUTATION_COUNT + = new CoordinatorStat("costComputeCount", "segment/balancer/compute/count"); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java index 33540800693..c2c77f02cbf 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java @@ -161,14 +161,14 @@ public class CoordinatorRunStatsTest ); expected.forEach( (duty, count) -> - stats.add(STAT_1, RowKey.builder().add(Dimension.DUTY, duty).build(), count) + stats.add(STAT_1, RowKey.of(Dimension.DUTY, duty), count) ); final Map actual = new HashMap<>(); stats.forEachStat( - (dimensionValues, stat, value) -> { + (stat, rowKey, value) -> { if (stat.equals(STAT_1)) { - actual.put(dimensionValues.get(Dimension.DUTY), value); + actual.put(rowKey.getValues().get(Dimension.DUTY), value); } } ); @@ -192,7 +192,7 @@ public class CoordinatorRunStatsTest debugStats.add(DEBUG_STAT_1, Key.TIER_1, 1); Assert.assertTrue(debugStats.hasStat(DEBUG_STAT_1)); - debugStats.addToDatasourceStat(DEBUG_STAT_2, "wiki", 1); + debugStats.add(DEBUG_STAT_2, RowKey.of(Dimension.DATASOURCE, "wiki"), 1); Assert.assertFalse(debugStats.hasStat(DEBUG_STAT_2)); } @@ -201,11 +201,11 @@ public class CoordinatorRunStatsTest */ private static class Key { - static final RowKey TIER_1 = RowKey.forTier("tier1"); - static final RowKey TIER_2 = RowKey.forTier("tier2"); + static final RowKey TIER_1 = RowKey.of(Dimension.TIER, "tier1"); + static final RowKey TIER_2 = RowKey.of(Dimension.TIER, "tier2"); - static final RowKey DUTY_1 = RowKey.builder().add(Dimension.DUTY, "duty1").build(); - static final RowKey DUTY_2 = RowKey.builder().add(Dimension.DUTY, "duty2").build(); + static final RowKey DUTY_1 = RowKey.of(Dimension.DUTY, "duty1"); + static final RowKey DUTY_2 = RowKey.of(Dimension.DUTY, "duty2"); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyTest.java index f4cad1cb5cd..23975352b33 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CachingCostBalancerStrategyTest.java @@ -38,6 +38,7 @@ import org.junit.Test; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Objects; import java.util.Random; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -102,7 +103,7 @@ public class CachingCostBalancerStrategyTest .findDestinationServerToMoveSegment(s, firstServer, serverHolderList); ServerHolder s2 = costBalancerStrategy .findDestinationServerToMoveSegment(s, firstServer, serverHolderList); - return (s1.getServer().getName().equals(s2.getServer().getName())) ? 0 : 1; + return Objects.equals(s1, s2) ? 0 : 1; } ) .sum(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java index 6580425764a..17d0a8716a1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java @@ -21,17 +21,19 @@ package org.apache.druid.server.coordinator.balancer; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.DruidServer; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.core.Event; -import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester; -import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.junit.After; import org.junit.Assert; @@ -61,7 +63,7 @@ public class CostBalancerStrategyTest @Before public void setup() { - balancerExecutor = new BlockingExecutorService("test-balance-exec-%d"); + balancerExecutor = Execs.singleThreaded("test-balance-exec-%d"); strategy = new CostBalancerStrategy(MoreExecutors.listeningDecorator(balancerExecutor)); serviceEmitter = new StubServiceEmitter("test-service", "host"); @@ -241,7 +243,7 @@ public class CostBalancerStrategyTest } @Test - public void testComputeCost() + public void testComputePlacementCost() { // Create segments for different granularities final List daySegments = @@ -265,7 +267,7 @@ public class CostBalancerStrategyTest .withNumPartitions(30) .eachOfSizeInMb(100); - // Distribute the segments randomly amongst 2 servers + // Distribute the segments randomly amongst 3 servers final List segments = new ArrayList<>(daySegments); segments.addAll(monthSegments); segments.addAll(yearSegments); @@ -284,35 +286,66 @@ public class CostBalancerStrategyTest server -> new ServerHolder(server.toImmutableDruidServer(), new LoadQueuePeonTester()) ).collect(Collectors.toList()); + final ServerHolder serverA = serverHolders.get(0); + final ServerHolder serverB = serverHolders.get(1); + final ServerHolder serverC = serverHolders.get(2); + // Verify costs for DAY, MONTH and YEAR segments - verifyServerCosts( - daySegments.get(0), - serverHolders, - 5191.500804, 8691.392080, 6418.467818 - ); - verifyServerCosts( - monthSegments.get(0), - serverHolders, - 301935.940609, 301935.940606, 304333.677669 - ); - verifyServerCosts( - yearSegments.get(0), - serverHolders, - 8468764.380437, 12098919.896931, 14501440.169452 - ); + final DataSegment daySegment = daySegments.get(0); + verifyPlacementCost(daySegment, serverA, 5191.500804); + verifyPlacementCost(daySegment, serverB, 8691.392080); + verifyPlacementCost(daySegment, serverC, 6418.467818); + + final DataSegment monthSegment = monthSegments.get(0); + verifyPlacementCost(monthSegment, serverA, 301935.940609); + verifyPlacementCost(monthSegment, serverB, 301935.940606); + verifyPlacementCost(monthSegment, serverC, 304333.677669); + + final DataSegment yearSegment = yearSegments.get(0); + verifyPlacementCost(yearSegment, serverA, 8468764.380437); + verifyPlacementCost(yearSegment, serverB, 12098919.896931); + verifyPlacementCost(yearSegment, serverC, 14501440.169452); // Verify costs for an ALL granularity segment - DataSegment allGranularitySegment = + final DataSegment allGranularitySegment = CreateDataSegments.ofDatasource(DS_WIKI) .forIntervals(1, Granularities.ALL) .eachOfSizeInMb(100).get(0); - verifyServerCosts( - allGranularitySegment, - serverHolders, - 1.1534173737329768e7, - 1.6340633534241956e7, - 1.9026400521582970e7 + verifyPlacementCost(allGranularitySegment, serverA, 1.1534173737329768e7); + verifyPlacementCost(allGranularitySegment, serverB, 1.6340633534241956e7); + verifyPlacementCost(allGranularitySegment, serverC, 1.9026400521582970e7); + } + + @Test + public void testGetAndResetStats() + { + final ServerHolder serverA = new ServerHolder( + createHistorical().toImmutableDruidServer(), + new LoadQueuePeonTester() ); + final ServerHolder serverB = new ServerHolder( + createHistorical().toImmutableDruidServer(), + new LoadQueuePeonTester() + ); + + final DataSegment segment = CreateDataSegments.ofDatasource(DS_WIKI).eachOfSizeInMb(100).get(0); + + // Verify that computation stats have been tracked + strategy.findServersToLoadSegment(segment, Arrays.asList(serverA, serverB)); + CoordinatorRunStats computeStats = strategy.getAndResetStats(); + + final RowKey rowKey = RowKey.with(Dimension.DATASOURCE, DS_WIKI) + .with(Dimension.DESCRIPTION, "LOAD") + .and(Dimension.TIER, "hot"); + Assert.assertEquals(1L, computeStats.get(Stats.Balancer.COMPUTATION_COUNT, rowKey)); + + long computeTime = computeStats.get(Stats.Balancer.COMPUTATION_TIME, rowKey); + Assert.assertTrue(computeTime >= 0 && computeTime <= 100); + Assert.assertFalse(computeStats.hasStat(Stats.Balancer.COMPUTATION_ERRORS)); + + // Verify that stats have been reset + computeStats = strategy.getAndResetStats(); + Assert.assertEquals(0, computeStats.rowCount()); } @Test @@ -334,42 +367,24 @@ public class CostBalancerStrategyTest ); } - @Test(timeout = 90_000L) - public void testFindServerRaisesAlertOnTimeout() + /** + * Verifies that the cost of placing the segment on the server is as expected. + * Also verifies that this cost is equal to the total joint cost of this segment + * with each segment currently on the server. + */ + private void verifyPlacementCost(DataSegment segment, ServerHolder server, double expectedCost) { - DataSegment segment = CreateDataSegments.ofDatasource(DS_WIKI) - .forIntervals(1, Granularities.DAY) - .startingAt("2012-10-24") - .eachOfSizeInMb(100).get(0); + double observedCost = strategy.computePlacementCost(segment, server); + Assert.assertEquals(expectedCost, observedCost, DELTA); - final LoadQueuePeonTester peon = new LoadQueuePeonTester(); - ServerHolder serverA = new ServerHolder(createHistorical().toImmutableDruidServer(), peon); - ServerHolder serverB = new ServerHolder(createHistorical().toImmutableDruidServer(), peon); - - strategy.findServersToLoadSegment(segment, Arrays.asList(serverA, serverB)); - - List events = serviceEmitter.getEvents(); - Assert.assertEquals(1, events.size()); - Assert.assertTrue(events.get(0) instanceof AlertEvent); - - AlertEvent alertEvent = (AlertEvent) events.get(0); - Assert.assertEquals( - "Cost balancer strategy timed out in action [findServersToLoadSegment]." - + " Try setting a higher value of 'balancerComputeThreads'.", - alertEvent.getDescription() - ); - } - - private void verifyServerCosts( - DataSegment segment, - List serverHolders, - double... expectedCosts - ) - { - for (int i = 0; i < serverHolders.size(); ++i) { - double observedCost = strategy.computeCost(segment, serverHolders.get(i), true); - Assert.assertEquals(expectedCosts[i], observedCost, DELTA); + double totalJointSegmentCost = 0; + for (DataSegment segmentOnServer : server.getServer().iterateAllSegments()) { + totalJointSegmentCost += CostBalancerStrategy.computeJointSegmentsCost(segment, segmentOnServer); } + if (server.isServingSegment(segment)) { + totalJointSegmentCost -= CostBalancerStrategy.computeJointSegmentsCost(segment, segment); + } + Assert.assertEquals(totalJointSegmentCost, observedCost, DELTA); } private void verifyJointSegmentsCost( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java index a506443e8ed..39dcc9ce50a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java @@ -37,7 +37,6 @@ import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; -import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.After; @@ -132,13 +131,8 @@ public class BalanceSegmentsTest @Test public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove() { - final ServerHolder serverHolder1 = createHolder(server1, false, segment1, segment2); - final ServerHolder serverHolder2 = createHolder(server2, true, segment3, segment4); - final ServerHolder serverHolder3 = createHolder(server3, false); - - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - expectFindDestinationAndReturn(strategy, serverHolder3); - EasyMock.replay(strategy); + final ServerHolder serverHolder1 = createHolder(server1, true, segment1, segment2, segment3, segment4); + final ServerHolder serverHolder2 = createHolder(server2, false); // ceil(3 * 0.6) = 2 segments from decommissioning servers CoordinatorDynamicConfig dynamicConfig = @@ -148,24 +142,21 @@ public class BalanceSegmentsTest .withDecommissioningMaxPercentOfMaxSegmentsToMove(60) .build(); DruidCoordinatorRuntimeParams params = - defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3) + defaultRuntimeParamsBuilder(serverHolder1, serverHolder2) .withDynamicConfigs(dynamicConfig) - .withBalancerStrategy(strategy) + .withBalancerStrategy(balancerStrategy) .withBroadcastDatasources(broadcastDatasources) .withSegmentAssignerUsing(loadQueueManager) .build(); CoordinatorRunStats stats = runBalancer(params); - EasyMock.verify(strategy); - - // 2 segments are moved from the decommissioning server and 1 from the active server - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")); - Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2")); - Set segmentsMoved = serverHolder3.getPeon().getSegmentsToLoad(); - Assert.assertTrue(segmentsMoved.contains(segment3)); - Assert.assertTrue(segmentsMoved.contains(segment4)); - Assert.assertTrue(segmentsMoved.contains(segment1) || segmentsMoved.contains(segment2)); + // 2 segments are moved from the decommissioning server + long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") + + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"); + Assert.assertEquals(2L, totalMoved); + Set segmentsMoved = serverHolder2.getPeon().getSegmentsToLoad(); + Assert.assertEquals(2, segmentsMoved.size()); } @Test @@ -220,33 +211,27 @@ public class BalanceSegmentsTest @Test public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning() { - final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2); - final ServerHolder serverHolder2 = createHolder(server2, segment3, segment4); - final ServerHolder serverHolder3 = createHolder(server3); - - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - expectFindDestinationAndReturn(strategy, serverHolder3); - EasyMock.replay(strategy); + final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2, segment3, segment4); + final ServerHolder serverHolder2 = createHolder(server2); CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig.builder() .withSmartSegmentLoading(false) - .withMaxSegmentsToMove(3) + .withMaxSegmentsToMove(4) .withDecommissioningMaxPercentOfMaxSegmentsToMove(9) .build(); DruidCoordinatorRuntimeParams params = - defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3) + defaultRuntimeParamsBuilder(serverHolder1, serverHolder2) .withDynamicConfigs(dynamicConfig) - .withBalancerStrategy(strategy) + .withBalancerStrategy(balancerStrategy) .withSegmentAssignerUsing(loadQueueManager) .build(); CoordinatorRunStats stats = runBalancer(params); - EasyMock.verify(strategy); long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"); - Assert.assertEquals(3L, totalMoved); - Assert.assertEquals(3, serverHolder3.getPeon().getSegmentsToLoad().size()); + Assert.assertEquals(2L, totalMoved); + Assert.assertEquals(2, serverHolder2.getPeon().getSegmentsToLoad().size()); } /** @@ -258,18 +243,13 @@ public class BalanceSegmentsTest final ServerHolder activeServer = createHolder(server1, false, allSegments); final ServerHolder decommissioningServer = createHolder(server2, true); - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - expectFindDestinationAndReturn(strategy, decommissioningServer); - EasyMock.replay(strategy); - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(activeServer, decommissioningServer) - .withBalancerStrategy(strategy) + .withBalancerStrategy(balancerStrategy) .withBroadcastDatasources(broadcastDatasources) .build(); CoordinatorRunStats stats = runBalancer(params); - EasyMock.verify(strategy); Assert.assertFalse(stats.hasStat(Stats.Segments.MOVED)); } @@ -279,22 +259,17 @@ public class BalanceSegmentsTest final ServerHolder decommissioningServer = createHolder(server1, true, allSegments); final ServerHolder activeServer = createHolder(server2); - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - expectFindDestinationAndReturn(strategy, activeServer); - EasyMock.replay(strategy); - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(decommissioningServer, activeServer) .withDynamicConfigs( CoordinatorDynamicConfig.builder() .withSmartSegmentLoading(false) .withMaxSegmentsToMove(3).build() ) - .withBalancerStrategy(strategy) + .withBalancerStrategy(balancerStrategy) .withBroadcastDatasources(broadcastDatasources) .build(); runBalancer(params); - EasyMock.verify(strategy); Assert.assertEquals(0, decommissioningServer.getPeon().getSegmentsToLoad().size()); Assert.assertEquals(3, activeServer.getPeon().getSegmentsToLoad().size()); } @@ -358,10 +333,6 @@ public class BalanceSegmentsTest final ServerHolder holder2 = createHolder(server2, segment3, segment4); final ServerHolder holder3 = createHolder(server3); - BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - expectFindDestinationAndReturn(strategy, holder3); - EasyMock.replay(strategy); - DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(holder1, holder2, holder3) .withDynamicConfigs( @@ -372,12 +343,11 @@ public class BalanceSegmentsTest .withPercentOfSegmentsToConsiderPerMove(40) .build() ) - .withBalancerStrategy(strategy) + .withBalancerStrategy(balancerStrategy) .withBroadcastDatasources(broadcastDatasources) .build(); CoordinatorRunStats stats = runBalancer(params); - EasyMock.verify(strategy); long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"); Assert.assertEquals(1L, totalMoved); @@ -463,17 +433,6 @@ public class BalanceSegmentsTest ); } - private void expectFindDestinationAndReturn(BalancerStrategy strategy, ServerHolder chosenServer) - { - EasyMock.expect( - strategy.findDestinationServerToMoveSegment( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - ) - ).andReturn(chosenServer).anyTimes(); - } - private DataSegment createHourlySegment(String datasource, DateTime start, String version) { return new DataSegment( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 14fcd05d005..7487d332d1f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -2037,7 +2037,7 @@ public class CompactSegmentsTest // all the datasources in the coordinator stats final AtomicInteger numDatasources = new AtomicInteger(); stats.forEachStat( - (dimensionValues, stat, value) -> { + (stat, rowKey, value) -> { if (stat.equals(Stats.Compaction.PENDING_BYTES) && (expectedRemainingSegments <= 0 || value == expectedRemainingSegments)) { numDatasources.incrementAndGet(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java index cd9c1e228da..46240a340de 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java @@ -76,6 +76,7 @@ public class RunRulesTest { private static final long SERVER_SIZE_10GB = 10L << 30; private static final String DATASOURCE = "test"; + private static final RowKey DATASOURCE_STAT_KEY = RowKey.of(Dimension.DATASOURCE, DATASOURCE); private LoadQueuePeon mockPeon; private RunRules ruleRunner; @@ -563,7 +564,7 @@ public class RunRulesTest .build(); CoordinatorRunStats stats = runDutyAndGetStats(params); - Assert.assertEquals(12L, stats.getDataSourceStat(Stats.Segments.DELETED, DATASOURCE)); + Assert.assertEquals(12L, stats.get(Stats.Segments.DELETED, DATASOURCE_STAT_KEY)); } @Test @@ -616,7 +617,7 @@ public class RunRulesTest CoordinatorRunStats stats = runDutyAndGetStats(params); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, "normal", DATASOURCE)); - Assert.assertEquals(12L, stats.getDataSourceStat(Stats.Segments.DELETED, DATASOURCE)); + Assert.assertEquals(12L, stats.get(Stats.Segments.DELETED, DATASOURCE_STAT_KEY)); EasyMock.verify(mockPeon); } @@ -662,7 +663,7 @@ public class RunRulesTest CoordinatorRunStats stats = runDutyAndGetStats(params); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, "normal", DATASOURCE)); - Assert.assertEquals(12L, stats.getDataSourceStat(Stats.Segments.DELETED, DATASOURCE)); + Assert.assertEquals(12L, stats.get(Stats.Segments.DELETED, DATASOURCE_STAT_KEY)); EasyMock.verify(mockPeon); } @@ -705,7 +706,7 @@ public class RunRulesTest CoordinatorRunStats stats = runDutyAndGetStats(params); Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); - Assert.assertEquals(12L, stats.getDataSourceStat(Stats.Segments.DELETED, DATASOURCE)); + Assert.assertEquals(12L, stats.get(Stats.Segments.DELETED, DATASOURCE_STAT_KEY)); EasyMock.verify(mockPeon); } @@ -1188,7 +1189,7 @@ public class RunRulesTest .build(); CoordinatorRunStats stats = runDutyAndGetStats(params); - final RowKey tierRowKey = RowKey.builder().add(Dimension.TIER, DruidServer.DEFAULT_TIER).build(); + final RowKey tierRowKey = RowKey.of(Dimension.TIER, DruidServer.DEFAULT_TIER); Assert.assertEquals( dataSegment.getSize() * numReplicants, stats.get(Stats.Tier.REQUIRED_CAPACITY, tierRowKey) @@ -1250,7 +1251,7 @@ public class RunRulesTest .build(); CoordinatorRunStats stats = runDutyAndGetStats(params); - final RowKey tierRowKey = RowKey.builder().add(Dimension.TIER, DruidServer.DEFAULT_TIER).build(); + final RowKey tierRowKey = RowKey.of(Dimension.TIER, DruidServer.DEFAULT_TIER); Assert.assertEquals( dataSegment.getSize() * numReplicants, stats.get(Stats.Tier.REQUIRED_CAPACITY, tierRowKey) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java index 668013f7ff3..c7e3b5b239f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordinator.rules; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; @@ -77,11 +76,8 @@ public class LoadRuleTest private ListeningExecutorService exec; private BalancerStrategy balancerStrategy; - private CachingCostBalancerStrategy cachingCostBalancerStrategy; - private SegmentLoadQueueManager loadQueueManager; private final boolean useRoundRobinAssignment; - private BalancerStrategy mockBalancerStrategy; private final AtomicInteger serverId = new AtomicInteger(); @@ -101,9 +97,6 @@ public class LoadRuleTest { exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "LoadRuleTest-%d")); balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - cachingCostBalancerStrategy = new CachingCostBalancerStrategy(ClusterCostCache.builder().build(), exec); - - mockBalancerStrategy = EasyMock.createMock(BalancerStrategy.class); loadQueueManager = new SegmentLoadQueueManager(null, null, null); } @@ -119,13 +112,7 @@ public class LoadRuleTest final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - - if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .times(2); - } - EasyMock.replay(mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon); DruidCluster druidCluster = DruidCluster .builder() @@ -140,14 +127,10 @@ public class LoadRuleTest Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI)); - EasyMock.verify(mockPeon, mockBalancerStrategy); + EasyMock.verify(mockPeon); } - private CoordinatorRunStats runRuleAndGetStats( - LoadRule rule, - DataSegment segment, - DruidCluster cluster - ) + private CoordinatorRunStats runRuleAndGetStats(LoadRule rule, DataSegment segment, DruidCluster cluster) { return runRuleAndGetStats(rule, segment, makeCoordinatorRuntimeParams(cluster, segment)); } @@ -160,7 +143,7 @@ public class LoadRuleTest { final StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); rule.run(segment, segmentAssigner); - return segmentAssigner.getStats(); + return params.getCoordinatorStats(); } private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParams( @@ -171,7 +154,7 @@ public class LoadRuleTest return DruidCoordinatorRuntimeParams .newBuilder(DateTimes.nowUtc()) .withDruidCluster(druidCluster) - .withBalancerStrategy(mockBalancerStrategy) + .withBalancerStrategy(balancerStrategy) .withUsedSegmentsInTest(usedSegments) .withDynamicConfigs( CoordinatorDynamicConfig.builder() @@ -189,16 +172,7 @@ public class LoadRuleTest final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - - LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - - final DataSegment segment = createDataSegment(DS_WIKI); - - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .anyTimes(); - - EasyMock.replay(mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon); ImmutableDruidServer server1 = createServer(Tier.T1).toImmutableDruidServer(); ImmutableDruidServer server2 = createServer(Tier.T1).toImmutableDruidServer(); @@ -207,6 +181,8 @@ public class LoadRuleTest .addTier(Tier.T1, new ServerHolder(server1, mockPeon), new ServerHolder(server2, mockPeon)) .build(); + final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); + final DataSegment segment = createDataSegment(DS_WIKI); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, segment.getDataSource())); @@ -223,7 +199,7 @@ public class LoadRuleTest Assert.assertFalse(statsAfterLoadPrimary.hasStat(Stats.Segments.ASSIGNED)); - EasyMock.verify(mockPeon, mockBalancerStrategy); + EasyMock.verify(mockPeon); } @Test @@ -233,16 +209,7 @@ public class LoadRuleTest final LoadQueuePeon emptyPeon = createEmptyPeon(); emptyPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - - LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - - final DataSegment segment = createDataSegment(DS_WIKI); - - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .anyTimes(); - - EasyMock.replay(emptyPeon, mockBalancerStrategy); + EasyMock.replay(emptyPeon); ImmutableDruidServer server1 = createServer(Tier.T1).toImmutableDruidServer(); ImmutableDruidServer server2 = createServer(Tier.T1).toImmutableDruidServer(); @@ -251,6 +218,8 @@ public class LoadRuleTest .addTier(Tier.T1, new ServerHolder(server1, emptyPeon), new ServerHolder(server2, emptyPeon)) .build(); + final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); + final DataSegment segment = createDataSegment(DS_WIKI); CoordinatorRunStats stats = runRuleAndGetStats( rule, segment, @@ -277,7 +246,7 @@ public class LoadRuleTest Assert.assertEquals(1L, statsAfterLoadPrimary.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); - EasyMock.verify(emptyPeon, mockBalancerStrategy); + EasyMock.verify(emptyPeon); } @Test @@ -286,16 +255,7 @@ public class LoadRuleTest final LoadQueuePeon emptyPeon = createEmptyPeon(); emptyPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - - LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - - final DataSegment segment = createDataSegment(DS_WIKI); - - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .anyTimes(); - - EasyMock.replay(emptyPeon, mockBalancerStrategy); + EasyMock.replay(emptyPeon); ImmutableDruidServer server1 = createServer(Tier.T1).toImmutableDruidServer(); ImmutableDruidServer server2 = createServer(Tier.T1).toImmutableDruidServer(); @@ -304,6 +264,8 @@ public class LoadRuleTest .addTier(Tier.T1, new ServerHolder(server1, emptyPeon), new ServerHolder(server2, emptyPeon)) .build(); + final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); + final DataSegment segment = createDataSegment(DS_WIKI); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); // Ensure that the segment is assigned to one of the historicals @@ -323,14 +285,12 @@ public class LoadRuleTest Assert.assertFalse(statsAfterLoadPrimary.hasStat(Stats.Segments.ASSIGNED)); - EasyMock.verify(emptyPeon, mockBalancerStrategy); + EasyMock.verify(emptyPeon); } @Test public void testLoadUsedSegmentsForAllSegmentGranularityAndCachingCostBalancerStrategy() { - LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - final List segments = CreateDataSegments.ofDatasource(DS_WIKI) .forIntervals(1, Granularities.ALL) @@ -338,21 +298,18 @@ public class LoadRuleTest .eachOfSizeInMb(100); final LoadQueuePeon loadingPeon = createLoadingPeon(segments.get(0), true); - loadingPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().once(); - - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(cachingCostBalancerStrategy) - .anyTimes(); - - EasyMock.replay(loadingPeon, mockBalancerStrategy); + EasyMock.replay(loadingPeon); DruidCluster druidCluster = DruidCluster .builder() .addTier(Tier.T1, createServerHolder(Tier.T1, loadingPeon, false)) .build(); + balancerStrategy = new CachingCostBalancerStrategy(ClusterCostCache.builder().build(), exec); + + LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); CoordinatorRunStats stats = runRuleAndGetStats( rule, segments.get(1), @@ -360,7 +317,7 @@ public class LoadRuleTest ); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); - EasyMock.verify(loadingPeon, mockBalancerStrategy); + EasyMock.verify(loadingPeon); } @Test @@ -369,10 +326,7 @@ public class LoadRuleTest final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - EasyMock.expect(mockBalancerStrategy.pickServersToDropSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .times(4); - EasyMock.replay(mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon); final DataSegment segment = createDataSegment(DS_WIKI); @@ -407,14 +361,7 @@ public class LoadRuleTest final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - - if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .times(1); - } - - EasyMock.replay(mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon); DruidCluster druidCluster = DruidCluster .builder() @@ -428,7 +375,7 @@ public class LoadRuleTest Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); - EasyMock.verify(mockPeon, mockBalancerStrategy); + EasyMock.verify(mockPeon); } @Test @@ -437,7 +384,7 @@ public class LoadRuleTest final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - EasyMock.replay(mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon); final DataSegment segment = createDataSegment(DS_WIKI); @@ -460,19 +407,12 @@ public class LoadRuleTest Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, DS_WIKI)); - EasyMock.verify(mockPeon, mockBalancerStrategy); + EasyMock.verify(mockPeon); } @Test public void testMaxLoadingQueueSize() { - if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .times(2); - } - EasyMock.replay(mockBalancerStrategy); - final LoadQueuePeonTester peon = new LoadQueuePeonTester(); final int maxSegmentsInQueue = 2; @@ -482,10 +422,7 @@ public class LoadRuleTest Tier.T1, new ServerHolder( createServer(Tier.T1).toImmutableDruidServer(), - peon, - false, - maxSegmentsInQueue, - 10 + peon, false, maxSegmentsInQueue, 10 ) ) .build(); @@ -497,7 +434,7 @@ public class LoadRuleTest DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams .newBuilder(DateTimes.nowUtc()) .withDruidCluster(druidCluster) - .withBalancerStrategy(mockBalancerStrategy) + .withBalancerStrategy(balancerStrategy) .withUsedSegmentsInTest(dataSegment1, dataSegment2, dataSegment3) .withDynamicConfigs( CoordinatorDynamicConfig.builder() @@ -517,8 +454,6 @@ public class LoadRuleTest Assert.assertEquals(1L, stats1.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, dataSegment1.getDataSource())); Assert.assertEquals(1L, stats2.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, dataSegment2.getDataSource())); Assert.assertEquals(0L, stats3.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, dataSegment3.getDataSource())); - - EasyMock.verify(mockBalancerStrategy); } /** @@ -530,14 +465,7 @@ public class LoadRuleTest { final LoadQueuePeon mockPeon1 = createEmptyPeon(); final LoadQueuePeon mockPeon2 = createOneCallPeonMock(); - - if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .times(1); - } - - EasyMock.replay(mockPeon1, mockPeon2, mockBalancerStrategy); + EasyMock.replay(mockPeon1, mockPeon2); DruidCluster druidCluster = DruidCluster .builder() @@ -550,7 +478,7 @@ public class LoadRuleTest CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI)); - EasyMock.verify(mockPeon1, mockPeon2, mockBalancerStrategy); + EasyMock.verify(mockPeon1, mockPeon2); } /** @@ -572,15 +500,6 @@ public class LoadRuleTest ServerHolder holder4 = createServerHolder(Tier.T2, mockPeon4, false); final DataSegment segment = createDataSegment(DS_WIKI); - if (!useRoundRobinAssignment) { - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(segment, ImmutableList.of(holder2))) - .andReturn(Collections.singletonList(holder2).iterator()); - EasyMock.expect(mockBalancerStrategy.findServersToLoadSegment(segment, ImmutableList.of(holder4, holder3))) - .andReturn(Arrays.asList(holder3, holder4).iterator()); - } - - EasyMock.replay(mockBalancerStrategy); - DruidCluster druidCluster = DruidCluster .builder() .addTier(Tier.T1, holder1, holder2) @@ -593,7 +512,7 @@ public class LoadRuleTest Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI)); - EasyMock.verify(mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy); + EasyMock.verify(mockPeon1, mockPeon2, mockPeon3, mockPeon4); } /** @@ -606,10 +525,7 @@ public class LoadRuleTest final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().times(2); - EasyMock.expect(mockBalancerStrategy.pickServersToDropSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .times(4); - EasyMock.replay(mockPeon, mockBalancerStrategy); + EasyMock.replay(mockPeon); final DataSegment segment1 = createDataSegment("foo1"); final DataSegment segment2 = createDataSegment("foo2"); @@ -652,10 +568,6 @@ public class LoadRuleTest final LoadQueuePeon mockPeon1 = new LoadQueuePeonTester(); final LoadQueuePeon mockPeon2 = new LoadQueuePeonTester(); final LoadQueuePeon mockPeon3 = new LoadQueuePeonTester(); - EasyMock.expect(mockBalancerStrategy.pickServersToDropSegment(EasyMock.anyObject(), EasyMock.anyObject())) - .andDelegateTo(balancerStrategy) - .times(4); - EasyMock.replay(mockBalancerStrategy); final DataSegment segment1 = createDataSegment(DS_WIKI); @@ -751,11 +663,11 @@ public class LoadRuleTest return mockPeon2; } - private ServerHolder createServerHolder(String tier, LoadQueuePeon mockPeon1, boolean isDecommissioning) + private ServerHolder createServerHolder(String tier, LoadQueuePeon loadQueuePeon, boolean isDecommissioning) { return new ServerHolder( createServer(tier).toImmutableDruidServer(), - mockPeon1, + loadQueuePeon, isDecommissioning ); } From c78d885b80a913deaf2c99600275df423daa81f0 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 27 Jun 2023 13:40:35 -0700 Subject: [PATCH 38/74] Cache parsed expressions and binding analysis in more places. (#14124) * Cache parsed expressions and binding analysis in more places. Main changes: 1) Cache parsed and analyzed expressions within PlannerContext for a single SQL query. 2) Cache parsed expressions together with input binding analysis using a new class AnalyzeExpr. This speeds up SQL planning, because SQL planning involves parsing analyzing the same expression strings over and over again. * Fixes. * Fix style. * Fix test. * Simplify: get rid of AnalyzedExpr, focus on caching. * Rename parse -> parseExpression. --- .../post/ExpressionPostAggregator.java | 33 ++-- .../query/filter/ExpressionDimFilter.java | 44 +++-- .../segment/join/JoinConditionAnalysis.java | 18 ++- .../virtual/ExpressionVirtualColumn.java | 58 +++++-- .../post/ExpressionPostAggregatorTest.java | 2 +- sql/pom.xml | 4 + .../builtin/ArrayConcatSqlAggregator.java | 2 +- .../builtin/ArraySqlAggregator.java | 2 +- .../aggregation/builtin/AvgSqlAggregator.java | 2 +- .../builtin/BitwiseSqlAggregator.java | 2 +- .../builtin/MultiColumnSqlAggregator.java | 153 ------------------ .../builtin/SimpleSqlAggregator.java | 2 +- .../builtin/StringSqlAggregator.java | 2 +- .../calcite/expression/DruidExpression.java | 29 ++-- .../sql/calcite/expression/Expressions.java | 27 ++-- .../expression/OperatorConversions.java | 2 +- .../ArrayContainsOperatorConversion.java | 3 +- .../ArrayOverlapOperatorConversion.java | 3 +- ...ExpressionDimFilterOperatorConversion.java | 3 +- .../builtin/CaseOperatorConversion.java | 12 +- .../builtin/CastOperatorConversion.java | 4 +- .../builtin/DateTruncOperatorConversion.java | 6 +- .../MultiValueStringOperatorConversions.java | 3 +- .../NestedDataOperatorConversions.java | 9 +- .../QueryLookupOperatorConversion.java | 2 +- .../RegexpExtractOperatorConversion.java | 4 +- .../builtin/TimeFloorOperatorConversion.java | 5 +- .../builtin/TruncateOperatorConversion.java | 2 +- .../sql/calcite/planner/ExpressionParser.java | 34 ++++ .../calcite/planner/ExpressionParserImpl.java | 56 +++++++ .../sql/calcite/planner/PlannerContext.java | 17 +- .../calcite/rel/DruidCorrelateUnnestRel.java | 2 +- .../sql/calcite/rel/DruidJoinQueryRel.java | 10 +- .../druid/sql/calcite/rel/DruidQuery.java | 6 +- .../druid/sql/calcite/rel/Grouping.java | 5 +- .../druid/sql/calcite/rel/Projection.java | 2 +- .../calcite/rel/VirtualColumnRegistry.java | 20 +-- .../sql/calcite/rule/DruidUnnestRule.java | 2 +- .../expression/ExpressionTestHelper.java | 14 +- .../druid/sql/calcite/rel/DruidQueryTest.java | 49 +++++- 40 files changed, 363 insertions(+), 292 deletions(-) delete mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MultiColumnSqlAggregator.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/planner/ExpressionParser.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/planner/ExpressionParserImpl.java diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregator.java index 4d4d8cd6831..9c62c9e7a5f 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregator.java @@ -67,7 +67,6 @@ public class ExpressionPostAggregator implements PostAggregator @Nullable private final String ordering; - private final ExprMacroTable macroTable; private final Map> finalizers; private final Expr.InputBindingInspector partialTypeInformation; @@ -76,7 +75,7 @@ public class ExpressionPostAggregator implements PostAggregator private final Supplier cacheKey; /** - * Constructor for serialization. + * Constructor for deserialization. */ @JsonCreator public ExpressionPostAggregator( @@ -90,16 +89,35 @@ public class ExpressionPostAggregator implements PostAggregator name, expression, ordering, - macroTable, Parser.lazyParse(expression, macroTable) ); } + /** + * Constructor for a pre-parsed expression. + */ + public ExpressionPostAggregator( + final String name, + final String expression, + @Nullable final String ordering, + final Expr parsed + ) + { + this( + name, + expression, + ordering, + () -> parsed + ); + } + + /** + * Constructor for a supplier of a pre-parsed expression. + */ private ExpressionPostAggregator( final String name, final String expression, @Nullable final String ordering, - final ExprMacroTable macroTable, final Supplier parsed ) { @@ -107,7 +125,6 @@ public class ExpressionPostAggregator implements PostAggregator name, expression, ordering, - macroTable, ImmutableMap.of(), InputBindings.nilBindings(), parsed, @@ -119,7 +136,6 @@ public class ExpressionPostAggregator implements PostAggregator final String name, final String expression, @Nullable final String ordering, - final ExprMacroTable macroTable, final Map> finalizers, final Expr.InputBindingInspector partialTypeInformation, final Supplier parsed, @@ -133,7 +149,6 @@ public class ExpressionPostAggregator implements PostAggregator this.ordering = ordering; // comparator should be specialized to output type ... someday this.comparator = ordering == null ? DEFAULT_COMPARATOR : Ordering.valueOf(ordering); - this.macroTable = macroTable; this.finalizers = finalizers; this.partialTypeInformation = partialTypeInformation; @@ -147,7 +162,6 @@ public class ExpressionPostAggregator implements PostAggregator }); } - @Override public Set getDependentFields() { @@ -208,7 +222,6 @@ public class ExpressionPostAggregator implements PostAggregator name, expression, ordering, - macroTable, finalizers, InputBindings.inspectorFromTypeMap(types), parsed, @@ -252,7 +265,7 @@ public class ExpressionPostAggregator implements PostAggregator * Ensures the following order: numeric > NaN > Infinite. * * The name may be referenced via Ordering.valueOf(String) in the constructor {@link - * ExpressionPostAggregator#ExpressionPostAggregator(String, String, String, ExprMacroTable, Map, Expr.InputBindingInspector, Supplier, Supplier)}. + * ExpressionPostAggregator#ExpressionPostAggregator(String, String, String, Map, Expr.InputBindingInspector, Supplier, Supplier)}. */ @SuppressWarnings("unused") numericFirst { diff --git a/processing/src/main/java/org/apache/druid/query/filter/ExpressionDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/ExpressionDimFilter.java index 32fc2c39dfd..f2f13bc0bfe 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/ExpressionDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/ExpressionDimFilter.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.RangeSet; @@ -45,6 +44,9 @@ public class ExpressionDimFilter extends AbstractOptimizableDimFilter implements @Nullable private final FilterTuning filterTuning; + /** + * Constructor for deserialization. + */ @JsonCreator public ExpressionDimFilter( @JsonProperty("expression") final String expression, @@ -52,20 +54,40 @@ public class ExpressionDimFilter extends AbstractOptimizableDimFilter implements @JacksonInject ExprMacroTable macroTable ) { - this.expression = expression; - this.filterTuning = filterTuning; - this.parsed = Parser.lazyParse(expression, macroTable); - this.cacheKey = Suppliers.memoize(() -> { - return new CacheKeyBuilder(DimFilterUtils.EXPRESSION_CACHE_ID) - .appendCacheable(parsed.get()) - .build(); - }); + this(expression, Parser.lazyParse(expression, macroTable), filterTuning); } - @VisibleForTesting + /** + * Constructor used in various tests that don't need to provide {@link FilterTuning}. + */ public ExpressionDimFilter(final String expression, ExprMacroTable macroTable) { - this(expression, null, macroTable); + this(expression, Parser.lazyParse(expression, macroTable), null); + } + + /** + * Constructor for already-parsed-and-analyzed expressions. + */ + public ExpressionDimFilter(final String expression, final Expr parsed, @Nullable final FilterTuning filterTuning) + { + this(expression, () -> parsed, filterTuning); + } + + private ExpressionDimFilter( + String expression, + Supplier parsed, + @Nullable FilterTuning filterTuning + ) + { + this.expression = expression; + this.parsed = parsed; + this.filterTuning = filterTuning; + this.cacheKey = Suppliers.memoize( + () -> + new CacheKeyBuilder(DimFilterUtils.EXPRESSION_CACHE_ID) + .appendCacheable(parsed.get()) + .build() + ); } @JsonProperty diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinConditionAnalysis.java b/processing/src/main/java/org/apache/druid/segment/join/JoinConditionAnalysis.java index 4a35998f324..2a33da22d13 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinConditionAnalysis.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinConditionAnalysis.java @@ -99,7 +99,23 @@ public class JoinConditionAnalysis final ExprMacroTable macroTable ) { - final Expr conditionExpr = Parser.parse(condition, macroTable); + return forExpression(condition, Parser.parse(condition, macroTable), rightPrefix); + } + + /** + * Analyze a join condition from a pre-parsed expression. + * + * @param condition the condition expression + * @param conditionExpr the parsed condition expression. Must match "condition". + * @param rightPrefix prefix for the right-hand side of the join; will be used to determine which identifiers in + * the condition come from the right-hand side and which come from the left-hand side + */ + public static JoinConditionAnalysis forExpression( + final String condition, + final Expr conditionExpr, + final String rightPrefix + ) + { final List equiConditions = new ArrayList<>(); final List nonEquiConditions = new ArrayList<>(); diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java index 80d3d4d48fb..f6488d4e9e3 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java @@ -63,6 +63,9 @@ public class ExpressionVirtualColumn implements VirtualColumn private final Supplier parsedExpression; private final Supplier cacheKey; + /** + * Constructor for deserialization. + */ @JsonCreator public ExpressionVirtualColumn( @JsonProperty("name") String name, @@ -71,28 +74,57 @@ public class ExpressionVirtualColumn implements VirtualColumn @JacksonInject ExprMacroTable macroTable ) { - this.name = Preconditions.checkNotNull(name, "name"); - this.expression = Preconditions.checkNotNull(expression, "expression"); - this.outputType = outputType; - this.parsedExpression = Parser.lazyParse(expression, macroTable); - this.cacheKey = makeCacheKeySupplier(); + this(name, expression, outputType, Parser.lazyParse(expression, macroTable)); } /** - * Constructor for creating an ExpressionVirtualColumn from a pre-parsed expression. + * Constructor for creating an ExpressionVirtualColumn from a pre-parsed-and-analyzed expression, where the original + * expression string is known. */ public ExpressionVirtualColumn( - String name, - Expr parsedExpression, - @Nullable ColumnType outputType + final String name, + final String expression, + final Expr parsedExpression, + @Nullable final ColumnType outputType + ) + { + this(name, expression, outputType, () -> parsedExpression); + } + + /** + * Constructor for creating an ExpressionVirtualColumn from a pre-parsed expression, where the original + * expression string is not known. + * + * This constructor leads to an instance where {@link #getExpression()} is the toString representation of the + * parsed expression, which is not necessarily a valid expression. Do not try to reparse it as an expression, as + * this will not work. + * + * If you know the original expression, use + * {@link ExpressionVirtualColumn#ExpressionVirtualColumn(String, String, Expr, ColumnType)} instead. + */ + public ExpressionVirtualColumn( + final String name, + final Expr parsedExpression, + @Nullable final ColumnType outputType + ) + { + this(name, parsedExpression.toString(), outputType, () -> parsedExpression); + } + + /** + * Private constructor used by the public ones. + */ + private ExpressionVirtualColumn( + final String name, + final String expression, + @Nullable final ColumnType outputType, + final Supplier parsedExpression ) { this.name = Preconditions.checkNotNull(name, "name"); - // Unfortunately this string representation can't be reparsed into the same expression, might be useful - // if the expression system supported that - this.expression = parsedExpression.toString(); + this.expression = Preconditions.checkNotNull(expression, "expression"); this.outputType = outputType; - this.parsedExpression = Suppliers.ofInstance(parsedExpression); + this.parsedExpression = parsedExpression; this.cacheKey = makeCacheKeySupplier(); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregatorTest.java index 98309709f39..4707a1380de 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregatorTest.java @@ -63,7 +63,7 @@ public class ExpressionPostAggregatorTest extends InitializedNullHandlingTest { EqualsVerifier.forClass(ExpressionPostAggregator.class) .usingGetClass() - .withIgnoredFields("macroTable", "finalizers", "parsed", "dependentFields", "cacheKey", "partialTypeInformation") + .withIgnoredFields("finalizers", "parsed", "dependentFields", "cacheKey", "partialTypeInformation") .verify(); } diff --git a/sql/pom.xml b/sql/pom.xml index d032a64f5e2..f301c575bf9 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -88,6 +88,10 @@ + + com.github.ben-manes.caffeine + caffeine + com.google.code.findbugs jsr305 diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java index 86aaf55c685..ed6652181eb 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java @@ -93,7 +93,7 @@ public class ArrayConcatSqlAggregator implements SqlAggregator maxSizeBytes = ((Number) RexLiteral.value(maxBytes)).intValue(); } final DruidExpression arg = Expressions.toDruidExpression(plannerContext, rowSignature, arguments.get(0)); - final ExprMacroTable macroTable = plannerContext.getExprMacroTable(); + final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable(); final String fieldName; final ColumnType druidType = Calcites.getValueTypeForRelDataTypeFull(aggregateCall.getType()); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java index ec914dac7f8..5136ed3c947 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java @@ -99,7 +99,7 @@ public class ArraySqlAggregator implements SqlAggregator // can't translate argument return null; } - final ExprMacroTable macroTable = plannerContext.getExprMacroTable(); + final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable(); final String fieldName; final String initialvalue; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/AvgSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/AvgSqlAggregator.java index 8d84ab30760..a938bdca0b8 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/AvgSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/AvgSqlAggregator.java @@ -94,7 +94,7 @@ public class AvgSqlAggregator implements SqlAggregator final DruidExpression arg = Iterables.getOnlyElement(arguments); - final ExprMacroTable macroTable = plannerContext.getExprMacroTable(); + final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable(); final ColumnType sumType; // Use 64-bit sum regardless of the type of the AVG aggregator. if (SqlTypeName.INT_TYPES.contains(aggregateCall.getType().getSqlTypeName())) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java index 52623a5d8c1..9d5023e8baf 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java @@ -142,7 +142,7 @@ public class BitwiseSqlAggregator implements SqlAggregator } final DruidExpression arg = arguments.get(0); - final ExprMacroTable macroTable = plannerContext.getExprMacroTable(); + final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable(); final String fieldName; if (arg.isDirectColumnAccess()) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MultiColumnSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MultiColumnSqlAggregator.java deleted file mode 100644 index 340e1870e3b..00000000000 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MultiColumnSqlAggregator.java +++ /dev/null @@ -1,153 +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.druid.sql.calcite.aggregation.builtin; - -import com.google.common.base.Preconditions; -import org.apache.calcite.rel.core.AggregateCall; -import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rex.RexBuilder; -import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.PostAggregator; -import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.sql.calcite.aggregation.Aggregation; -import org.apache.druid.sql.calcite.aggregation.Aggregations; -import org.apache.druid.sql.calcite.aggregation.SqlAggregator; -import org.apache.druid.sql.calcite.expression.DruidExpression; -import org.apache.druid.sql.calcite.planner.Calcites; -import org.apache.druid.sql.calcite.planner.PlannerContext; -import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; - -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.List; - -/** - * Abstraction for simple multi-column post aggregators like greatest, least - */ -public abstract class MultiColumnSqlAggregator implements SqlAggregator -{ - /** - * Useful Abstraction for passing field information to subclasses from shared parent methods - */ - protected static class FieldInfo - { - final String fieldName; - final String expression; - - private FieldInfo(String fieldName, String expression) - { - this.fieldName = fieldName; - this.expression = expression; - } - - public static FieldInfo fromFieldName(String fieldName) - { - return new FieldInfo(fieldName, null); - } - - public static FieldInfo fromExpression(String expression) - { - return new FieldInfo(null, expression); - } - } - - @Nullable - @Override - public Aggregation toDruidAggregation( - final PlannerContext plannerContext, - final RowSignature rowSignature, - final VirtualColumnRegistry virtualColumnRegistry, - final RexBuilder rexBuilder, - final String name, - final AggregateCall aggregateCall, - final Project project, - final List existingAggregations, - final boolean finalizeAggregations - ) - { - if (aggregateCall.isDistinct()) { - return null; - } - - final List arguments = Aggregations.getArgumentsForSimpleAggregator( - rexBuilder, - plannerContext, - rowSignature, - aggregateCall, - project - ); - - if (arguments == null) { - return null; - } - - final ExprMacroTable macroTable = plannerContext.getExprMacroTable(); - final List fieldInfoList = new ArrayList<>(); - - // Convert arguments to concise field information - for (DruidExpression argument : arguments) { - if (argument.isDirectColumnAccess()) { - fieldInfoList.add(FieldInfo.fromFieldName(argument.getDirectColumn())); - } else { - fieldInfoList.add(FieldInfo.fromExpression(argument.getExpression())); - } - } - Preconditions.checkArgument(!fieldInfoList.isEmpty(), "FieldInfoList should not be empty"); - return getAggregation(name, aggregateCall, macroTable, fieldInfoList); - } - - private Aggregation getAggregation( - String name, - AggregateCall aggregateCall, - ExprMacroTable macroTable, - List fieldInfoList - ) - { - final ColumnType valueType = Calcites.getColumnTypeForRelDataType(aggregateCall.getType()); - List aggregatorFactories = new ArrayList<>(); - List postAggregators = new ArrayList<>(); - - // Delegate aggregator factory construction to subclasses for provided fields. - // Create corresponding field access post aggregators. - int id = 0; - for (FieldInfo fieldInfo : fieldInfoList) { - String prefixedName = Calcites.makePrefixedName(name, String.valueOf(id++)); - postAggregators.add(new FieldAccessPostAggregator(null, prefixedName)); - aggregatorFactories.add(createAggregatorFactory(valueType, prefixedName, fieldInfo, macroTable)); - } - // Delegate final post aggregator construction to subclasses by passing the above aggregators. - final PostAggregator finalPostAggregator = createFinalPostAggregator(valueType, name, postAggregators); - return Aggregation.create(aggregatorFactories, finalPostAggregator); - } - - abstract AggregatorFactory createAggregatorFactory( - ColumnType valueType, - String prefixedName, - FieldInfo fieldInfo, - ExprMacroTable macroTable); - - abstract PostAggregator createFinalPostAggregator( - ColumnType valueType, - String name, - List postAggregators); -} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SimpleSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SimpleSqlAggregator.java index 3ac68483611..01782668663 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SimpleSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SimpleSqlAggregator.java @@ -84,7 +84,7 @@ public abstract class SimpleSqlAggregator implements SqlAggregator } final DruidExpression arg = Iterables.getOnlyElement(arguments); - final ExprMacroTable macroTable = plannerContext.getExprMacroTable(); + final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable(); final String fieldName; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java index 4411ebd33cd..a6044d53e36 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java @@ -128,7 +128,7 @@ public class StringSqlAggregator implements SqlAggregator maxSizeBytes = ((Number) RexLiteral.value(maxBytes)).intValue(); } final DruidExpression arg = arguments.get(0); - final ExprMacroTable macroTable = plannerContext.getExprMacroTable(); + final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable(); final String initialvalue = "[]"; final ColumnType elementType = ColumnType.STRING; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/DruidExpression.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/DruidExpression.java index e5252fbe5a9..b06f3e402f2 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/DruidExpression.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/DruidExpression.java @@ -26,11 +26,10 @@ import com.google.common.io.BaseEncoding; import com.google.common.primitives.Chars; import org.apache.calcite.rel.type.RelDataType; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.Parser; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.sql.calcite.planner.ExpressionParser; import javax.annotation.Nullable; import java.util.ArrayList; @@ -48,7 +47,7 @@ import java.util.function.Function; * * When added to {@link org.apache.druid.sql.calcite.rel.VirtualColumnRegistry} whenever used by projections, filters, * aggregators, or other query components, these will be converted into native virtual columns using - * {@link #toVirtualColumn(String, ColumnType, ExprMacroTable)} + * {@link #toVirtualColumn(String, ColumnType, ExpressionParser)} * * Approximate expression structure is retained in the {@link #arguments}, which when fed into the * {@link ExpressionGenerator} that all {@link DruidExpression} must be created with will produce the final String @@ -375,30 +374,22 @@ public class DruidExpression return arguments; } - /** - * Compile the {@link DruidExpression} into a string and parse it into a native Druid {@link Expr} - */ - public Expr parse(final ExprMacroTable macroTable) - { - return Parser.parse(expression.get(), macroTable); - } - public VirtualColumn toVirtualColumn( final String name, final ColumnType outputType, - final ExprMacroTable macroTable + final ExpressionParser parser ) { - return virtualColumnCreator.create(name, outputType, expression.get(), macroTable); + return virtualColumnCreator.create(name, outputType, expression.get(), parser); } public VirtualColumn toExpressionVirtualColumn( final String name, final ColumnType outputType, - final ExprMacroTable macroTable + final ExpressionParser parser ) { - return DEFAULT_VIRTUAL_COLUMN_BUILDER.create(name, outputType, expression.get(), macroTable); + return DEFAULT_VIRTUAL_COLUMN_BUILDER.create(name, outputType, expression.get(), parser); } public NodeType getType() @@ -413,7 +404,7 @@ public class DruidExpression * supplied by other means. * * This value is not currently used other than for tracking the types of the {@link DruidExpression} tree. The - * value passed to {@link #toVirtualColumn(String, ColumnType, ExprMacroTable)} will instead be whatever type "hint" + * value passed to {@link #toVirtualColumn(String, ColumnType, ExpressionParser)} will instead be whatever type "hint" * was specified when the expression was added to the {@link org.apache.druid.sql.calcite.rel.VirtualColumnRegistry}. */ @Nullable @@ -597,15 +588,15 @@ public class DruidExpression @FunctionalInterface public interface VirtualColumnCreator { - VirtualColumn create(String name, ColumnType outputType, String expression, ExprMacroTable macroTable); + VirtualColumn create(String name, ColumnType outputType, String expression, ExpressionParser parser); } public static class ExpressionVirtualColumnCreator implements VirtualColumnCreator { @Override - public VirtualColumn create(String name, ColumnType outputType, String expression, ExprMacroTable macroTable) + public VirtualColumn create(String name, ColumnType outputType, String expression, ExpressionParser parser) { - return new ExpressionVirtualColumn(name, expression, outputType, macroTable); + return new ExpressionVirtualColumn(name, expression, parser.parse(expression), outputType); } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java index 21c23bf2906..d429f1fd8a9 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java @@ -37,8 +37,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.Parser; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.expression.TimestampFloorExprMacro; import org.apache.druid.query.extraction.ExtractionFn; @@ -58,6 +56,7 @@ import org.apache.druid.sql.calcite.filtration.BoundRefKey; import org.apache.druid.sql.calcite.filtration.Bounds; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.ExpressionParser; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; import org.apache.druid.sql.calcite.table.RowSignatures; @@ -598,7 +597,8 @@ public class Expressions } // Special handling for filters on FLOOR(__time TO granularity). - final Granularity queryGranularity = toQueryGranularity(lhsExpression, plannerContext.getExprMacroTable()); + final Granularity queryGranularity = + toQueryGranularity(lhsExpression, plannerContext.getExpressionParser()); if (queryGranularity != null) { // lhs is FLOOR(__time TO granularity); rhs must be a timestamp final long rhsMillis = Calcites.calciteDateTimeLiteralToJoda(rhs, plannerContext.getTimeZone()).getMillis(); @@ -713,9 +713,16 @@ public class Expressions ) { final DruidExpression druidExpression = toDruidExpression(plannerContext, rowSignature, rexNode); - return druidExpression != null - ? new ExpressionDimFilter(druidExpression.getExpression(), plannerContext.getExprMacroTable()) - : null; + + if (druidExpression != null) { + return new ExpressionDimFilter( + druidExpression.getExpression(), + plannerContext.parseExpression(druidExpression.getExpression()), + null + ); + } + + return null; } /** @@ -725,9 +732,9 @@ public class Expressions * @return granularity or null if not possible */ @Nullable - public static Granularity toQueryGranularity(final DruidExpression expression, final ExprMacroTable macroTable) + public static Granularity toQueryGranularity(final DruidExpression expression, final ExpressionParser parser) { - final TimestampFloorExprMacro.TimestampFloorExpr expr = asTimestampFloorExpr(expression, macroTable); + final TimestampFloorExprMacro.TimestampFloorExpr expr = asTimestampFloorExpr(expression, parser); if (expr == null) { return null; @@ -746,10 +753,10 @@ public class Expressions @Nullable public static TimestampFloorExprMacro.TimestampFloorExpr asTimestampFloorExpr( final DruidExpression expression, - final ExprMacroTable macroTable + final ExpressionParser parser ) { - final Expr expr = Parser.parse(expression.getExpression(), macroTable); + final Expr expr = parser.parse(expression.getExpression()); if (expr instanceof TimestampFloorExprMacro.TimestampFloorExpr) { return (TimestampFloorExprMacro.TimestampFloorExpr) expr; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java index 6e3543d7372..e1c81c32ce5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java @@ -303,7 +303,7 @@ public class OperatorConversions postAggregatorVisitor.getOutputNamePrefix() + postAggregatorVisitor.getAndIncrementCounter(), druidExpression.getExpression(), null, - plannerContext.getExprMacroTable() + plannerContext.parseExpression(druidExpression.getExpression()) ); } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java index 02f56ffdaf1..36c5fd77d09 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java @@ -29,7 +29,6 @@ import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.InputBindings; -import org.apache.druid.math.expr.Parser; import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.segment.column.RowSignature; @@ -96,7 +95,7 @@ public class ArrayContainsOperatorConversion extends BaseExpressionDimFilterOper final DruidExpression rightExpr = druidExpressions.get(1); if (leftExpr.isSimpleExtraction() && !(leftExpr.isDirectColumnAccess() && leftExpr.getDruidType() != null && leftExpr.getDruidType().isArray())) { - Expr expr = Parser.parse(rightExpr.getExpression(), plannerContext.getExprMacroTable()); + Expr expr = plannerContext.parseExpression(rightExpr.getExpression()); // To convert this expression filter into an And of Selector filters, we need to extract all array elements. // For now, we can optimize only when rightExpr is a literal because there is no way to extract the array elements // by traversing the Expr. Note that all implementations of Expr are defined as package-private classes in a diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java index fbf4f654d5f..b0e25e14737 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java @@ -29,7 +29,6 @@ import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.InputBindings; -import org.apache.druid.math.expr.Parser; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.segment.column.RowSignature; @@ -112,7 +111,7 @@ public class ArrayOverlapOperatorConversion extends BaseExpressionDimFilterOpera return toExpressionFilter(plannerContext, getDruidFunctionName(), druidExpressions); } - Expr expr = Parser.parse(complexExpr.getExpression(), plannerContext.getExprMacroTable()); + Expr expr = plannerContext.parseExpression(complexExpr.getExpression()); if (expr.isLiteral()) { // Evaluate the expression to take out the array elements. // We can safely pass null if the expression is literal. diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BaseExpressionDimFilterOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BaseExpressionDimFilterOperatorConversion.java index a1f6dc36bbb..d24ea8ea891 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BaseExpressionDimFilterOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BaseExpressionDimFilterOperatorConversion.java @@ -50,7 +50,8 @@ public abstract class BaseExpressionDimFilterOperatorConversion extends DirectOp return new ExpressionDimFilter( filterExpr, - plannerContext.getExprMacroTable() + plannerContext.parseExpression(filterExpr), + null ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CaseOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CaseOperatorConversion.java index 392a965fa15..39d5c952d92 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CaseOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CaseOperatorConversion.java @@ -142,9 +142,17 @@ public class CaseOperatorConversion implements SqlOperatorConversion // if either then or else filters produced a native filter (that wasn't just another expression filter) // make sure we have filters for both sides by filling in the gaps with expression filter if (thenFilter != null && !(thenFilter instanceof ExpressionDimFilter) && elseFilter == null) { - elseFilter = new ExpressionDimFilter(elseExpression.getExpression(), plannerContext.getExprMacroTable()); + elseFilter = new ExpressionDimFilter( + elseExpression.getExpression(), + plannerContext.parseExpression(elseExpression.getExpression()), + null + ); } else if (thenFilter == null && elseFilter != null && !(elseFilter instanceof ExpressionDimFilter)) { - thenFilter = new ExpressionDimFilter(thenExpression.getExpression(), plannerContext.getExprMacroTable()); + thenFilter = new ExpressionDimFilter( + thenExpression.getExpression(), + plannerContext.parseExpression(thenExpression.getExpression()), + null + ); } if (thenFilter != null && elseFilter != null) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CastOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CastOperatorConversion.java index bb0b60db569..7779d1b1747 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CastOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CastOperatorConversion.java @@ -118,7 +118,7 @@ public class CastOperatorConversion implements SqlOperatorConversion return TimeFloorOperatorConversion.applyTimestampFloor( typeCastExpression, new PeriodGranularity(Period.days(1), null, plannerContext.getTimeZone()), - plannerContext.getExprMacroTable() + plannerContext ); } else { return typeCastExpression; @@ -148,7 +148,7 @@ public class CastOperatorConversion implements SqlOperatorConversion return TimeFloorOperatorConversion.applyTimestampFloor( timestampExpression, new PeriodGranularity(Period.days(1), null, plannerContext.getTimeZone()), - plannerContext.getExprMacroTable() + plannerContext ); } else if (toType == SqlTypeName.TIMESTAMP) { return timestampExpression; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java index 361de42862b..1fa5e8791d1 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java @@ -30,7 +30,6 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.Parser; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.OperatorConversions; @@ -91,10 +90,7 @@ public class DateTruncOperatorConversion implements SqlOperatorConversion rexNode, inputExpressions -> { final DruidExpression arg = inputExpressions.get(1); - final Expr truncTypeExpr = Parser.parse( - inputExpressions.get(0).getExpression(), - plannerContext.getExprMacroTable() - ); + final Expr truncTypeExpr = plannerContext.parseExpression(inputExpressions.get(0).getExpression()); if (!truncTypeExpr.isLiteral()) { throw new IAE("Operator[%s] truncType must be a literal", calciteOperator().getName()); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringOperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringOperatorConversions.java index d64ca9b1920..0ad0f545670 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringOperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringOperatorConversions.java @@ -32,7 +32,6 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.InputBindings; -import org.apache.druid.math.expr.Parser; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.virtual.ListFilteredVirtualColumn; @@ -333,7 +332,7 @@ public class MultiValueStringOperatorConversions return null; } - Expr expr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable()); + Expr expr = plannerContext.parseExpression(druidExpressions.get(1).getExpression()); // the right expression must be a literal array for this to work, since we need the values of the column if (!expr.isLiteral()) { return null; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java index 949b8b8879d..77a33e266c3 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java @@ -46,7 +46,6 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.InputBindings; -import org.apache.druid.math.expr.Parser; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.nested.NestedPathFinder; @@ -191,7 +190,7 @@ public class NestedDataOperatorConversions return null; } - final Expr pathExpr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable()); + final Expr pathExpr = plannerContext.parseExpression(druidExpressions.get(1).getExpression()); if (!pathExpr.isLiteral()) { return null; } @@ -371,7 +370,7 @@ public class NestedDataOperatorConversions return null; } - final Expr pathExpr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable()); + final Expr pathExpr = plannerContext.parseExpression(druidExpressions.get(1).getExpression()); if (!pathExpr.isLiteral()) { return null; } @@ -493,7 +492,7 @@ public class NestedDataOperatorConversions return null; } - final Expr pathExpr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable()); + final Expr pathExpr = plannerContext.parseExpression(druidExpressions.get(1).getExpression()); if (!pathExpr.isLiteral()) { return null; } @@ -661,7 +660,7 @@ public class NestedDataOperatorConversions return null; } - final Expr pathExpr = Parser.parse(druidExpressions.get(1).getExpression(), plannerContext.getExprMacroTable()); + final Expr pathExpr = plannerContext.parseExpression(druidExpressions.get(1).getExpression()); if (!pathExpr.isLiteral()) { return null; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java index aeb89d92028..18c58691d28 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java @@ -72,7 +72,7 @@ public class QueryLookupOperatorConversion implements SqlOperatorConversion StringUtils.toLowerCase(calciteOperator().getName()), inputExpressions -> { final DruidExpression arg = inputExpressions.get(0); - final Expr lookupNameExpr = inputExpressions.get(1).parse(plannerContext.getExprMacroTable()); + final Expr lookupNameExpr = plannerContext.parseExpression(inputExpressions.get(1).getExpression()); if (arg.isSimpleExtraction() && lookupNameExpr.isLiteral()) { return arg.getSimpleExtraction().cascade( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java index 804e5cad05d..d131e5a66a0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java @@ -66,9 +66,9 @@ public class RegexpExtractOperatorConversion implements SqlOperatorConversion StringUtils.toLowerCase(calciteOperator().getName()), inputExpressions -> { final DruidExpression arg = inputExpressions.get(0); - final Expr patternExpr = inputExpressions.get(1).parse(plannerContext.getExprMacroTable()); + final Expr patternExpr = plannerContext.parseExpression(inputExpressions.get(1).getExpression()); final Expr indexExpr = inputExpressions.size() > 2 - ? inputExpressions.get(2).parse(plannerContext.getExprMacroTable()) + ? plannerContext.parseExpression(inputExpressions.get(2).getExpression()) : null; if (arg.isSimpleExtraction() && patternExpr.isLiteral() && (indexExpr == null || indexExpr.isLiteral())) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java index b0b5b69acc7..d1a4644c55c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java @@ -32,7 +32,6 @@ import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.java.util.common.granularity.PeriodGranularity; -import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.expression.TimestampFloorExprMacro; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -69,7 +68,7 @@ public class TimeFloorOperatorConversion implements SqlOperatorConversion public static DruidExpression applyTimestampFloor( final DruidExpression input, final PeriodGranularity granularity, - final ExprMacroTable macroTable + final PlannerContext plannerContext ) { Preconditions.checkNotNull(input, "input"); @@ -79,7 +78,7 @@ public class TimeFloorOperatorConversion implements SqlOperatorConversion if (granularity.getPeriod().equals(Period.days(1))) { final TimestampFloorExprMacro.TimestampFloorExpr floorExpr = Expressions.asTimestampFloorExpr( input, - macroTable + plannerContext.getExpressionParser() ); if (floorExpr != null) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TruncateOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TruncateOperatorConversion.java index d891823dd15..ca6cdc8b6e6 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TruncateOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TruncateOperatorConversion.java @@ -62,7 +62,7 @@ public class TruncateOperatorConversion implements SqlOperatorConversion inputExpressions -> { final DruidExpression arg = inputExpressions.get(0); final Expr digitsExpr = inputExpressions.size() > 1 - ? inputExpressions.get(1).parse(plannerContext.getExprMacroTable()) + ? plannerContext.parseExpression(inputExpressions.get(1).getExpression()) : null; final String factorString; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExpressionParser.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExpressionParser.java new file mode 100644 index 00000000000..83d99082f27 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExpressionParser.java @@ -0,0 +1,34 @@ +/* + * 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.druid.sql.calcite.planner; + +import org.apache.druid.math.expr.Expr; +import org.apache.druid.math.expr.Parser; + +/** + * Like {@link Parser#parse}, but allows caching via {@link ExpressionParserImpl}. + */ +public interface ExpressionParser +{ + /** + * Parse a string into a flattened {@link Expr}. Parsing is eager and the results may be cached. + */ + Expr parse(String expr); +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExpressionParserImpl.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExpressionParserImpl.java new file mode 100644 index 00000000000..640de445f31 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExpressionParserImpl.java @@ -0,0 +1,56 @@ +/* + * 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.druid.sql.calcite.planner; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Weigher; +import org.apache.druid.math.expr.Expr; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.math.expr.Parser; + +/** + * Member of {@link PlannerToolbox} that caches the result of {@link Parser#parse}. + */ +public class ExpressionParserImpl implements ExpressionParser +{ + /** + * Maximum total expression length stored in the {@link #cache}, measured in characters of unparsed expressions. + */ + private static final int MAX_EXPRESSION_WEIGHT = 1_000_000; + + private final Cache cache; + private final ExprMacroTable macroTable; + + public ExpressionParserImpl(final ExprMacroTable macroTable) + { + this.cache = Caffeine.newBuilder() + .maximumWeight(MAX_EXPRESSION_WEIGHT) + .weigher((Weigher) (key, value) -> key.length()) + .build(); + this.macroTable = macroTable; + } + + @Override + public Expr parse(final String expression) + { + return cache.get(expression, k -> Parser.parse(expression, macroTable)); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 5c36ef72b66..abf837bb9c5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -32,7 +32,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Numbers; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.math.expr.Expr; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.join.JoinableFactoryWrapper; @@ -89,6 +89,7 @@ public class PlannerContext public static final String DATA_CTX_AUTHENTICATION_RESULT = "authenticationResult"; private final PlannerToolbox plannerToolbox; + private final ExpressionParser expressionParser; private final String sql; private final PlannerConfig plannerConfig; private final DateTime localNow; @@ -126,6 +127,7 @@ public class PlannerContext ) { this.plannerToolbox = plannerToolbox; + this.expressionParser = new ExpressionParserImpl(plannerToolbox.exprMacroTable()); this.sql = sql; this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig"); this.engine = engine; @@ -229,10 +231,17 @@ public class PlannerContext return plannerToolbox; } - // Deprecated: prefer using the toolbox - public ExprMacroTable getExprMacroTable() + public ExpressionParser getExpressionParser() { - return plannerToolbox.exprMacroTable(); + return expressionParser; + } + + /** + * Equivalent to {@link ExpressionParser#parse(String)} on {@link #getExpressionParser()}. + */ + public Expr parseExpression(final String expr) + { + return expressionParser.parse(expr); } // Deprecated: prefer using the toolbox diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java index 889d30648f0..04a1e8805bc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java @@ -190,7 +190,7 @@ public class DruidCorrelateUnnestRel extends DruidRel expressionToUnnest.toVirtualColumn( correlateRowSignature.getColumnName(correlateRowSignature.size() - 1), Calcites.getColumnTypeForRelDataType(rexNodeToUnnest.getType()), - getPlannerContext().getExprMacroTable() + getPlannerContext().getExpressionParser() ), unnestFilterOnDataSource ), diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java index 4ea14c3a741..9c3aa479fc2 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java @@ -48,6 +48,7 @@ import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; @@ -171,7 +172,7 @@ public class DruidJoinQueryRel extends DruidRel VirtualColumnRegistry virtualColumnRegistry = VirtualColumnRegistry.create( prefixSignaturePair.rhs, - getPlannerContext().getExprMacroTable(), + getPlannerContext().getExpressionParser(), getPlannerContext().getPlannerConfig().isForceExpressionVirtualColumns() ); getPlannerContext().setJoinExpressionVirtualColumnRegistry(virtualColumnRegistry); @@ -198,10 +199,13 @@ public class DruidJoinQueryRel extends DruidRel leftDataSource, rightDataSource, prefixSignaturePair.lhs, - condition.getExpression(), + JoinConditionAnalysis.forExpression( + condition.getExpression(), + getPlannerContext().parseExpression(condition.getExpression()), + prefixSignaturePair.lhs + ), toDruidJoinType(joinRel.getJoinType()), getDimFilter(getPlannerContext(), leftSignature, leftFilter), - getPlannerContext().getExprMacroTable(), getPlannerContext().getJoinableFactoryWrapper() ), prefixSignaturePair.rhs, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index d0fcacf5df2..3b80dda9dcc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -199,7 +199,7 @@ public class DruidQuery if (virtualColumnRegistry == null) { virtualColumnRegistry = VirtualColumnRegistry.create( sourceRowSignature, - plannerContext.getExprMacroTable(), + plannerContext.getExpressionParser(), plannerContext.getPlannerConfig().isForceExpressionVirtualColumns() ); } @@ -1043,7 +1043,7 @@ public class DruidQuery final DimensionExpression dimensionExpression = Iterables.getOnlyElement(grouping.getDimensions()); queryGranularity = Expressions.toQueryGranularity( dimensionExpression.getDruidExpression(), - plannerContext.getExprMacroTable() + plannerContext.getExpressionParser() ); if (queryGranularity == null) { @@ -1315,7 +1315,7 @@ public class DruidQuery for (DimensionExpression dimensionExpression : grouping.getDimensions()) { Granularity granularity = Expressions.toQueryGranularity( dimensionExpression.getDruidExpression(), - plannerContext.getExprMacroTable() + plannerContext.getExpressionParser() ); if (granularity == null || !canUseQueryGranularity(dataSource, filtration, granularity)) { // Can't, or won't, convert this dimension to a query granularity. diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Grouping.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Grouping.java index d6931fca47b..6627cc082ac 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Grouping.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Grouping.java @@ -27,7 +27,6 @@ import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.core.Project; import org.apache.calcite.util.ImmutableBitSet; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.math.expr.Parser; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; @@ -235,8 +234,8 @@ public class Grouping for (int i = 0; i < dimensions.size(); i++) { final DimensionExpression dimension = dimensions.get(i); - if (Parser.parse(dimension.getDruidExpression().getExpression(), plannerContext.getExprMacroTable()) - .isLiteral() && !aggregateProjectBits.get(i)) { + if (plannerContext.parseExpression(dimension.getDruidExpression().getExpression()).isLiteral() + && !aggregateProjectBits.get(i)) { droppedDimensions = true; newDimIndexes[i] = -1; } else { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java index e66bdd9270c..ef2730fb074 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java @@ -175,7 +175,7 @@ public class Projection postAggregatorVisitor.getOutputNamePrefix() + postAggregatorVisitor.getAndIncrementCounter(), postAggregatorExpression.getExpression(), null, - plannerContext.getExprMacroTable() + plannerContext.parseExpression(postAggregatorExpression.getExpression()) ); postAggregatorVisitor.addPostAgg(postAggregator); rowOrder.add(postAggregator.getName()); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java index 0e8e8f7ae18..b44be8b4389 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java @@ -21,13 +21,13 @@ package org.apache.druid.sql.calcite.rel; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.ExpressionParser; import org.apache.druid.sql.calcite.planner.PlannerContext; import javax.annotation.Nullable; @@ -47,7 +47,7 @@ import java.util.stream.Collectors; */ public class VirtualColumnRegistry { - private final ExprMacroTable macroTable; + private final ExpressionParser expressionParser; private final RowSignature baseRowSignature; private final Map virtualColumnsByExpression; private final Map virtualColumnsByName; @@ -57,14 +57,14 @@ public class VirtualColumnRegistry private VirtualColumnRegistry( RowSignature baseRowSignature, - ExprMacroTable macroTable, + ExpressionParser expressionParser, String virtualColumnPrefix, boolean forceExpressionVirtualColumns, Map virtualColumnsByExpression, Map virtualColumnsByName ) { - this.macroTable = macroTable; + this.expressionParser = expressionParser; this.baseRowSignature = baseRowSignature; this.virtualColumnPrefix = virtualColumnPrefix; this.virtualColumnsByExpression = virtualColumnsByExpression; @@ -74,13 +74,13 @@ public class VirtualColumnRegistry public static VirtualColumnRegistry create( final RowSignature rowSignature, - final ExprMacroTable macroTable, + final ExpressionParser expressionParser, final boolean forceExpressionVirtualColumns ) { return new VirtualColumnRegistry( rowSignature, - macroTable, + expressionParser, Calcites.findUnusedPrefixForDigits("v", rowSignature.getColumnNames()), forceExpressionVirtualColumns, new HashMap<>(), @@ -158,8 +158,8 @@ public class VirtualColumnRegistry DruidExpression expression = registeredColumn.getExpression(); ColumnType columnType = registeredColumn.getTypeHint(); return forceExpressionVirtualColumns - ? expression.toExpressionVirtualColumn(virtualColumnName, columnType, macroTable) - : expression.toVirtualColumn(virtualColumnName, columnType, macroTable); + ? expression.toExpressionVirtualColumn(virtualColumnName, columnType, expressionParser) + : expression.toVirtualColumn(virtualColumnName, columnType, expressionParser); } @Nullable @@ -187,7 +187,7 @@ public class VirtualColumnRegistry final ColumnCapabilities virtualCapabilities = virtualColumn.getValue().getExpression().toVirtualColumn( columnName, typeHint, - macroTable + expressionParser ).capabilities(baseSignature, columnName); // fall back to type hint @@ -300,7 +300,7 @@ public class VirtualColumnRegistry * Wrapper class for a {@link DruidExpression} and the output {@link ColumnType} "hint" that callers can specify when * adding a virtual column with {@link #getOrCreateVirtualColumnForExpression(DruidExpression, RelDataType)} or * {@link #getOrCreateVirtualColumnForExpression(DruidExpression, ColumnType)}. This "hint" will be passed into - * {@link DruidExpression#toVirtualColumn(String, ColumnType, ExprMacroTable)}. + * {@link DruidExpression#toVirtualColumn(String, ColumnType, ExpressionParser)}. * * The type hint might be different than {@link DruidExpression#getDruidType()} since that value is the captured value * of {@link org.apache.calcite.rex.RexNode#getType()} converted to the Druid type system, while callers might still diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidUnnestRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidUnnestRule.java index aa3e7848306..a5f3453f19a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidUnnestRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidUnnestRule.java @@ -154,7 +154,7 @@ public class DruidUnnestRule extends RelOptRule } // Evaluate the expression. It's a constant, so no bindings are needed. - final Expr parsedExpression = expression.parse(plannerContext.getExprMacroTable()); + final Expr parsedExpression = plannerContext.parseExpression(expression.getExpression()); final ExprEval eval = parsedExpression.eval(InputBindings.nilBindings()); final List rows = new ArrayList<>(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java index 307f7dcb3e1..5a72d01b423 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java @@ -35,8 +35,6 @@ import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; -import org.apache.druid.math.expr.Parser; -import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.RowAdapters; @@ -51,6 +49,7 @@ import org.apache.druid.sql.calcite.planner.CalciteRulesManager; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.CatalogResolver; import org.apache.druid.sql.calcite.planner.DruidTypeSystem; +import org.apache.druid.sql.calcite.planner.ExpressionParserImpl; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.planner.PlannerToolbox; @@ -336,8 +335,9 @@ class ExpressionTestHelper Assert.assertEquals("Expression for: " + rexNode, expectedExpression.getExpression(), expression.getExpression()); } - ExprEval result = Parser.parse(expression.getExpression(), PLANNER_CONTEXT.getExprMacroTable()) - .eval(expressionBindings); + ExprEval result = PLANNER_CONTEXT.parseExpression(expression.getExpression()) + + .eval(expressionBindings); Assert.assertEquals("Result for: " + rexNode, expectedResult, result.value()); } @@ -351,7 +351,11 @@ class ExpressionTestHelper ) { final RexNode rexNode = rexBuilder.makeCall(op, exprs); - final VirtualColumnRegistry virtualColumnRegistry = VirtualColumnRegistry.create(rowSignature, TestExprMacroTable.INSTANCE, false); + final VirtualColumnRegistry virtualColumnRegistry = VirtualColumnRegistry.create( + rowSignature, + new ExpressionParserImpl(PLANNER_TOOLBOX.exprMacroTable()), + false + ); final DimFilter filter = Expressions.toFilter(PLANNER_CONTEXT, rowSignature, virtualColumnRegistry, rexNode); Assert.assertEquals("Filter for: " + rexNode, expectedFilter, filter); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java index 844569d4ab6..dfc6229681a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java @@ -35,6 +35,7 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinType; import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.planner.ExpressionParserImpl; import org.apache.druid.sql.calcite.util.CalciteTests; import org.joda.time.Interval; import org.junit.Assert; @@ -64,7 +65,11 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, selectorFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + VirtualColumnRegistry.create( + RowSignature.empty(), + new ExpressionParserImpl(TestExprMacroTable.INSTANCE), + false + ), CalciteTests.createJoinableFactoryWrapper() ); verify(pair, dataSource, selectorFilter, Intervals.ETERNITY); @@ -77,7 +82,11 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, filterWithInterval, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + VirtualColumnRegistry.create( + RowSignature.empty(), + new ExpressionParserImpl(TestExprMacroTable.INSTANCE), + false + ), CalciteTests.createJoinableFactoryWrapper() ); verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200)); @@ -90,7 +99,11 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, filterWithInterval, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + VirtualColumnRegistry.create( + RowSignature.empty(), + new ExpressionParserImpl(TestExprMacroTable.INSTANCE), + false + ), CalciteTests.createJoinableFactoryWrapper() ); verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200)); @@ -104,7 +117,11 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + VirtualColumnRegistry.create( + RowSignature.empty(), + new ExpressionParserImpl(TestExprMacroTable.INSTANCE), + false + ), CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); @@ -118,7 +135,11 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + VirtualColumnRegistry.create( + RowSignature.empty(), + new ExpressionParserImpl(TestExprMacroTable.INSTANCE), + false + ), CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); @@ -132,7 +153,11 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + VirtualColumnRegistry.create( + RowSignature.empty(), + new ExpressionParserImpl(TestExprMacroTable.INSTANCE), + false + ), CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); @@ -146,7 +171,11 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + VirtualColumnRegistry.create( + RowSignature.empty(), + new ExpressionParserImpl(TestExprMacroTable.INSTANCE), + false + ), CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); @@ -165,7 +194,11 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, queryFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + VirtualColumnRegistry.create( + RowSignature.empty(), + new ExpressionParserImpl(TestExprMacroTable.INSTANCE), + false + ), CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(150, 200)); From 2cfb00b1de928a48a1c61bc04c10cdbccab33866 Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Tue, 27 Jun 2023 16:35:15 -0700 Subject: [PATCH 39/74] Add missing `isNull()` implementation to `FilteredAggregator` (#14465) --- .../query/aggregation/FilteredAggregator.java | 6 + .../aggregation/FilteredAggregatorTest.java | 406 ++++++++++-------- .../TestNullableDoubleColumnSelector.java | 4 - .../TestNullableFloatColumnSelector.java | 4 - .../TestNullableLongColumnSelector.java | 4 - .../WrappingDimensionSelectorTest.java | 3 +- 6 files changed, 231 insertions(+), 196 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregator.java index 6ffea5b70f4..edb6b8a6017 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregator.java @@ -40,6 +40,12 @@ public class FilteredAggregator implements Aggregator } } + @Override + public boolean isNull() + { + return delegate.isNull(); + } + @Override public Object get() { diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java index 7c1d8095947..da2e7be83d0 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java @@ -45,46 +45,221 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.DimensionSelectorUtils; import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.TestNullableFloatColumnSelector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.SingleIndexedInt; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; import java.util.Arrays; -public class FilteredAggregatorTest +public class FilteredAggregatorTest extends InitializedNullHandlingTest { - private void aggregate(TestFloatColumnSelector selector, FilteredAggregator agg) - { - agg.aggregate(); - selector.increment(); - } - @Test public void testAggregate() { - final float[] values = {0.15f, 0.27f}; - final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); - - FilteredAggregatorFactory factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), + final Float[] values = {0.15f, 0.27f}; + final TestNullableFloatColumnSelector selector = new TestNullableFloatColumnSelector(values); + final FilteredAggregatorFactory factory = new FilteredAggregatorFactory( + new FloatSumAggregatorFactory("billy", "value"), new SelectorDimFilter("dim", "a", null) ); - FilteredAggregator agg = (FilteredAggregator) factory.factorize(makeColumnSelector(selector)); - - double expectedFirst = new Float(values[0]).doubleValue(); - double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; - double expectedThird = expectedSecond; - - assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird); + final Float[] expectedVals = {values[0], values[0] + values[1]}; + validateFilteredAggs(factory, selector, expectedVals); } - private ColumnSelectorFactory makeColumnSelector(final TestFloatColumnSelector selector) + @Test + public void testAggregateWithNullVals() + { + final Float[] values = {0.15f, null, 0.27f}; + final TestNullableFloatColumnSelector selector = new TestNullableFloatColumnSelector(values); + + final FilteredAggregatorFactory factory = new FilteredAggregatorFactory( + new FloatSumAggregatorFactory("billy", "value"), + new SelectorDimFilter("dim", "a", null) + ); + + final Float[] expectedValues = {values[0], values[0], values[0] + values[2]}; + validateFilteredAggs(factory, selector, expectedValues); + } + + @Test + public void testAggregateOnlyWithNulls() + { + final Float[] values = {null, null}; + final TestNullableFloatColumnSelector selector = new TestNullableFloatColumnSelector(values); + final FilteredAggregatorFactory factory = new FilteredAggregatorFactory( + new FloatSumAggregatorFactory("billy", "value"), + new SelectorDimFilter("dim", "a", null) + ); + + final Float[] expectedValues = {NullHandling.defaultFloatValue(), NullHandling.defaultFloatValue()}; + validateFilteredAggs(factory, selector, expectedValues); + } + + + @Test + public void testAggregateWithNotFilter() + { + final Float[] values = {0.15f, 0.27f}; + final TestNullableFloatColumnSelector selector = new TestNullableFloatColumnSelector(values); + final FilteredAggregatorFactory factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new NotDimFilter(new SelectorDimFilter("dim", "b", null)) + ); + + final Float[] expectedValues = {values[0], values[0] + values[1]}; + validateFilteredAggs(factory, selector, expectedValues); + } + + @Test + public void testAggregateWithOrFilter() + { + final Float[] values = {0.15f, 0.27f, 0.14f}; + final TestNullableFloatColumnSelector selector = new TestNullableFloatColumnSelector(values); + + final FilteredAggregatorFactory factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new OrDimFilter(Lists.newArrayList( + new SelectorDimFilter("dim", "a", null), + new SelectorDimFilter("dim", "b", null) + )) + ); + + final Float[] expectedValues = {values[0], values[0] + values[1], values[0] + values[1] + values[2]}; + validateFilteredAggs(factory, selector, expectedValues); + } + + @Test + public void testAggregateWithAndFilter() + { + final Float[] values = {0.15f, 0.27f}; + final TestNullableFloatColumnSelector selector = new TestNullableFloatColumnSelector(values); + final FilteredAggregatorFactory factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new AndDimFilter(Lists.newArrayList( + new NotDimFilter(new SelectorDimFilter("dim", "b", null)), + new SelectorDimFilter("dim", "a", null) + )) + ); + + final Float[] expectedValues = {values[0], values[0] + values[1]}; + validateFilteredAggs(factory, selector, expectedValues); + } + + @Test + public void testAggregateWithPredicateFilters2() + { + final Float[] values = {0.15f, 0.27f, null}; + final Float[] expectedValues = {values[0], values[0] + values[1], values[0] + values[1]}; + TestNullableFloatColumnSelector selector; + FilteredAggregatorFactory factory; + + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new BoundDimFilter("dim", "a", "a", false, false, true, null, StringComparators.ALPHANUMERIC) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new RegexDimFilter("dim", "a", null) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new SearchQueryDimFilter("dim", new ContainsSearchQuerySpec("a", true), null) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + + String jsFn = "function(x) { return(x === 'a') }"; + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new JavaScriptDimFilter("dim", jsFn, null, JavaScriptConfig.getEnabledInstance()) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + } + + @Test + public void testAggregateWithExtractionFns() + { + final Float[] values = {0.15f, null, 0.27f, null, 0.13f}; + final Float[] expectedValues = { + values[0], + values[0], + values[0] + values[2], + values[0] + values[2], + values[0] + values[2] + values[4] + }; + + TestNullableFloatColumnSelector selector; + FilteredAggregatorFactory factory; + + String extractionJsFn = "function(str) { return str + 'AARDVARK'; }"; + ExtractionFn extractionFn = new JavaScriptExtractionFn( + extractionJsFn, + false, + JavaScriptConfig.getEnabledInstance() + ); + + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new SelectorDimFilter("dim", "aAARDVARK", extractionFn) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new InDimFilter("dim", Arrays.asList("NOT-aAARDVARK", "FOOBAR", "aAARDVARK"), extractionFn) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new BoundDimFilter("dim", "aAARDVARK", "aAARDVARK", false, false, true, extractionFn, + StringComparators.ALPHANUMERIC + ) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new RegexDimFilter("dim", "aAARDVARK", extractionFn) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new SearchQueryDimFilter("dim", new ContainsSearchQuerySpec("aAARDVARK", true), extractionFn) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + + String jsFn = "function(x) { return(x === 'aAARDVARK') }"; + factory = new FilteredAggregatorFactory( + new DoubleSumAggregatorFactory("billy", "value"), + new JavaScriptDimFilter("dim", jsFn, extractionFn, JavaScriptConfig.getEnabledInstance()) + ); + selector = new TestNullableFloatColumnSelector(values); + validateFilteredAggs(factory, selector, expectedValues); + } + + private ColumnSelectorFactory makeColumnSelector(final TestNullableFloatColumnSelector selector) { return new ColumnSelectorFactory() @@ -102,11 +277,7 @@ public class FilteredAggregatorTest public IndexedInts getRow() { SingleIndexedInt row = new SingleIndexedInt(); - if (selector.getIndex() % 3 == 2) { - row.setValue(1); - } else { - row.setValue(0); - } + row.setValue(0); return row; } @@ -216,174 +387,43 @@ public class FilteredAggregatorTest }; } - private void assertValues(FilteredAggregator agg, TestFloatColumnSelector selector, double... expectedVals) + private void aggregate( + final TestNullableFloatColumnSelector selector, + final FilteredAggregator agg + ) { - Assert.assertEquals(NullHandling.defaultDoubleValue(), agg.get()); - Assert.assertEquals(NullHandling.defaultDoubleValue(), agg.get()); - Assert.assertEquals(NullHandling.defaultDoubleValue(), agg.get()); - for (double expectedVal : expectedVals) { - aggregate(selector, agg); - Assert.assertEquals(expectedVal, agg.get()); - Assert.assertEquals(expectedVal, agg.get()); - Assert.assertEquals(expectedVal, agg.get()); - } - } - - @Test - public void testAggregateWithNotFilter() - { - final float[] values = {0.15f, 0.27f}; - final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); - - FilteredAggregatorFactory factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new NotDimFilter(new SelectorDimFilter("dim", "b", null)) - ); - - validateFilteredAggs(factory, values, selector); - } - - @Test - public void testAggregateWithOrFilter() - { - final float[] values = {0.15f, 0.27f, 0.14f}; - final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); - - FilteredAggregatorFactory factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new OrDimFilter(Lists.newArrayList(new SelectorDimFilter("dim", "a", null), new SelectorDimFilter("dim", "b", null))) - ); - - FilteredAggregator agg = (FilteredAggregator) factory.factorize( - makeColumnSelector(selector) - ); - - double expectedFirst = new Float(values[0]).doubleValue(); - double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; - double expectedThird = expectedSecond + new Float(values[2]).doubleValue(); - assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird); - } - - @Test - public void testAggregateWithAndFilter() - { - final float[] values = {0.15f, 0.27f}; - final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); - - FilteredAggregatorFactory factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new AndDimFilter(Lists.newArrayList(new NotDimFilter(new SelectorDimFilter("dim", "b", null)), new SelectorDimFilter("dim", "a", null)))); - - validateFilteredAggs(factory, values, selector); - } - - @Test - public void testAggregateWithPredicateFilters() - { - final float[] values = {0.15f, 0.27f}; - TestFloatColumnSelector selector; - FilteredAggregatorFactory factory; - - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new BoundDimFilter("dim", "a", "a", false, false, true, null, StringComparators.ALPHANUMERIC) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); - - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new RegexDimFilter("dim", "a", null) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); - - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new SearchQueryDimFilter("dim", new ContainsSearchQuerySpec("a", true), null) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); - - String jsFn = "function(x) { return(x === 'a') }"; - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new JavaScriptDimFilter("dim", jsFn, null, JavaScriptConfig.getEnabledInstance()) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); - } - - @Test - public void testAggregateWithExtractionFns() - { - final float[] values = {0.15f, 0.27f}; - TestFloatColumnSelector selector; - FilteredAggregatorFactory factory; - - String extractionJsFn = "function(str) { return str + 'AARDVARK'; }"; - ExtractionFn extractionFn = new JavaScriptExtractionFn(extractionJsFn, false, JavaScriptConfig.getEnabledInstance()); - - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new SelectorDimFilter("dim", "aAARDVARK", extractionFn) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); - - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new InDimFilter("dim", Arrays.asList("NOT-aAARDVARK", "FOOBAR", "aAARDVARK"), extractionFn) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); - - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new BoundDimFilter("dim", "aAARDVARK", "aAARDVARK", false, false, true, extractionFn, - StringComparators.ALPHANUMERIC - ) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); - - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new RegexDimFilter("dim", "aAARDVARK", extractionFn) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); - - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new SearchQueryDimFilter("dim", new ContainsSearchQuerySpec("aAARDVARK", true), extractionFn) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); - - String jsFn = "function(x) { return(x === 'aAARDVARK') }"; - factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new JavaScriptDimFilter("dim", jsFn, extractionFn, JavaScriptConfig.getEnabledInstance()) - ); - selector = new TestFloatColumnSelector(values); - validateFilteredAggs(factory, values, selector); + agg.aggregate(); + selector.increment(); } private void validateFilteredAggs( - FilteredAggregatorFactory factory, - float[] values, - TestFloatColumnSelector selector + final FilteredAggregatorFactory factory, + final TestNullableFloatColumnSelector selector, + final Float[] expectedValues ) { FilteredAggregator agg = (FilteredAggregator) factory.factorize( makeColumnSelector(selector) ); - double expectedFirst = new Float(values[0]).doubleValue(); - double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; - double expectedThird = expectedSecond; + // Validate state before any aggregation + if (NullHandling.sqlCompatible()) { + Assert.assertTrue(agg.isNull()); + Assert.assertNull(agg.get()); + } else { + Assert.assertFalse(agg.isNull()); + Assert.assertEquals(0.0f, agg.getFloat(), 0.001); + } - assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird); + for (Float expectedValue : expectedValues) { + aggregate(selector, agg); + if (expectedValue == null) { + Assert.assertTrue(agg.isNull()); + Assert.assertNull(agg.get()); + } else { + Assert.assertFalse(agg.isNull()); + Assert.assertEquals(expectedValue, agg.getFloat(), 0.001); + } + } } } diff --git a/processing/src/test/java/org/apache/druid/segment/TestNullableDoubleColumnSelector.java b/processing/src/test/java/org/apache/druid/segment/TestNullableDoubleColumnSelector.java index 609ce1e01dc..39530701278 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestNullableDoubleColumnSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/TestNullableDoubleColumnSelector.java @@ -25,10 +25,6 @@ public class TestNullableDoubleColumnSelector extends TestDoubleColumnSelector { private final Double[] doubles; - static { - NullHandling.initializeForTests(); - } - private int index = 0; public TestNullableDoubleColumnSelector(Double[] doubles) diff --git a/processing/src/test/java/org/apache/druid/segment/TestNullableFloatColumnSelector.java b/processing/src/test/java/org/apache/druid/segment/TestNullableFloatColumnSelector.java index 19b49610de4..a96b31cb92e 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestNullableFloatColumnSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/TestNullableFloatColumnSelector.java @@ -26,10 +26,6 @@ public class TestNullableFloatColumnSelector extends TestFloatColumnSelector private final Float[] floats; - static { - NullHandling.initializeForTests(); - } - private int index = 0; public TestNullableFloatColumnSelector(Float[] floats) diff --git a/processing/src/test/java/org/apache/druid/segment/TestNullableLongColumnSelector.java b/processing/src/test/java/org/apache/druid/segment/TestNullableLongColumnSelector.java index 5e27e995c78..a30b6c9e353 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestNullableLongColumnSelector.java +++ b/processing/src/test/java/org/apache/druid/segment/TestNullableLongColumnSelector.java @@ -25,10 +25,6 @@ public class TestNullableLongColumnSelector extends TestLongColumnSelector { private final Long[] longs; - static { - NullHandling.initializeForTests(); - } - private int index = 0; public TestNullableLongColumnSelector(Long[] longs) diff --git a/processing/src/test/java/org/apache/druid/segment/WrappingDimensionSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/WrappingDimensionSelectorTest.java index b208b96f81f..8ba3425807d 100644 --- a/processing/src/test/java/org/apache/druid/segment/WrappingDimensionSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/WrappingDimensionSelectorTest.java @@ -20,10 +20,11 @@ package org.apache.druid.segment; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; -public class WrappingDimensionSelectorTest +public class WrappingDimensionSelectorTest extends InitializedNullHandlingTest { @Test public void testLongWrappingDimensionSelector() From 0335aaa279d95063c6a172cf3f1ffdf6e46f592b Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 28 Jun 2023 10:14:04 +0530 Subject: [PATCH 40/74] Add query results directory and prevent the auto cleaner from cleaning it (#14446) Adds support for automatic cleaning of a "query-results" directory in durable storage. This directory will be cleaned up only if the task id is not known to the overlord. This will allow the storage of query results after the task has finished running. --- docs/multi-stage-query/reference.md | 2 +- .../msq/indexing/DurableStorageCleaner.java | 15 ++++-- .../indexing/DurableStorageCleanerTest.java | 49 +++++++++++++++++-- .../druid/frame/util/DurableStorageUtils.java | 41 ++++++++++++++-- .../frame/util/DurableStorageUtilsTest.java | 39 ++++++++++++--- 5 files changed, 129 insertions(+), 17 deletions(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index ec6a5b1543d..5bbe935f1ee 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -365,7 +365,7 @@ For detailed information about the settings related to durable storage, see [Dur ### Use durable storage for queries -When you run a query, include the context parameter `durableShuffleStorage` and set it to `true`. +When you run a query, include the context parameter `durableShuffleStorage` and set it to `true`. For queries where you want to use fault tolerance for workers, set `faultTolerance` to `true`, which automatically sets `durableShuffleStorage` to `true`. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java index f2b5be699b3..195b1e26f89 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java @@ -88,13 +88,22 @@ public class DurableStorageCleaner implements OverlordDuty .map(TaskRunnerWorkItem::getTaskId) .map(DurableStorageUtils::getControllerDirectory) .collect(Collectors.toSet()); + Set knownTaskIds = taskRunner.getKnownTasks() + .stream() + .map(TaskRunnerWorkItem::getTaskId) + .map(DurableStorageUtils::getControllerDirectory) + .collect(Collectors.toSet()); Set filesToRemove = new HashSet<>(); while (allFiles.hasNext()) { String currentFile = allFiles.next(); - String taskIdFromPathOrEmpty = DurableStorageUtils.getControllerTaskIdWithPrefixFromPath(currentFile); - if (taskIdFromPathOrEmpty != null && !taskIdFromPathOrEmpty.isEmpty()) { - if (runningTaskIds.contains(taskIdFromPathOrEmpty)) { + String nextDirName = DurableStorageUtils.getNextDirNameWithPrefixFromPath(currentFile); + if (nextDirName != null && !nextDirName.isEmpty()) { + if (runningTaskIds.contains(nextDirName)) { + // do nothing + } else if (DurableStorageUtils.QUERY_RESULTS_DIR.equals(nextDirName) + && DurableStorageUtils.isQueryResultFileActive(currentFile, knownTaskIds)) { + // query results should not be cleaned even if the task has finished running // do nothing } else { filesToRemove.add(currentFile); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java index 0f674739b28..7f7bf4fbfd1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java @@ -31,6 +31,7 @@ import org.apache.druid.storage.StorageConnector; import org.easymock.Capture; import org.easymock.EasyMock; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.util.Collection; @@ -45,19 +46,25 @@ public class DurableStorageCleanerTest private static final TaskRunnerWorkItem TASK_RUNNER_WORK_ITEM = EasyMock.mock(TaskRunnerWorkItem.class); private static final String TASK_ID = "dummyTaskId"; private static final String STRAY_DIR = "strayDirectory"; + private DurableStorageCleaner durableStorageCleaner; - @Test - public void testRun() throws Exception + @Before + public void setUp() { - EasyMock.reset(TASK_RUNNER, TASK_RUNNER_WORK_ITEM, STORAGE_CONNECTOR); + EasyMock.reset(TASK_RUNNER, TASK_RUNNER_WORK_ITEM, STORAGE_CONNECTOR, TASK_MASTER); DurableStorageCleanerConfig durableStorageCleanerConfig = new DurableStorageCleanerConfig(); durableStorageCleanerConfig.delaySeconds = 1L; durableStorageCleanerConfig.enabled = true; - DurableStorageCleaner durableStorageCleaner = new DurableStorageCleaner( + durableStorageCleaner = new DurableStorageCleaner( durableStorageCleanerConfig, STORAGE_CONNECTOR, () -> TASK_MASTER ); + } + + @Test + public void testRun() throws Exception + { EasyMock.expect(STORAGE_CONNECTOR.listDir(EasyMock.anyString())) .andReturn(ImmutableList.of(DurableStorageUtils.getControllerDirectory(TASK_ID), STRAY_DIR) .stream() @@ -68,15 +75,49 @@ public class DurableStorageCleanerTest EasyMock.expect((Collection) TASK_RUNNER.getRunningTasks()) .andReturn(ImmutableList.of(TASK_RUNNER_WORK_ITEM)) .anyTimes(); + EasyMock.expect((Collection) TASK_RUNNER.getKnownTasks()) + .andReturn(ImmutableList.of(TASK_RUNNER_WORK_ITEM)) + .anyTimes(); EasyMock.expect(TASK_MASTER.getTaskRunner()).andReturn(Optional.of(TASK_RUNNER)).anyTimes(); Capture> capturedArguments = EasyMock.newCapture(); STORAGE_CONNECTOR.deleteFiles(EasyMock.capture(capturedArguments)); EasyMock.expectLastCall().once(); EasyMock.replay(TASK_MASTER, TASK_RUNNER, TASK_RUNNER_WORK_ITEM, STORAGE_CONNECTOR); + durableStorageCleaner.run(); + Assert.assertEquals(Sets.newHashSet(STRAY_DIR), capturedArguments.getValue()); } + @Test + public void testRunExcludesQueryDirectory() throws Exception + { + final String resultPath = DurableStorageUtils.QUERY_RESULTS_DIR + "/" + DurableStorageUtils.getControllerDirectory(TASK_ID) + "/results.json"; + final String intermediateFilesPath = DurableStorageUtils.getControllerDirectory(TASK_ID) + "/intermediate.frame"; + EasyMock.expect(STORAGE_CONNECTOR.listDir(EasyMock.anyString())) + .andReturn(ImmutableList.of(resultPath, STRAY_DIR, intermediateFilesPath) + .stream() + .iterator()) + .anyTimes(); + EasyMock.expect(TASK_MASTER.getTaskRunner()).andReturn(Optional.of(TASK_RUNNER)).anyTimes(); + EasyMock.expect(TASK_RUNNER_WORK_ITEM.getTaskId()).andReturn(TASK_ID) + .anyTimes(); + EasyMock.expect((Collection) TASK_RUNNER.getRunningTasks()) + .andReturn(ImmutableList.of()) + .anyTimes(); + EasyMock.expect((Collection) TASK_RUNNER.getKnownTasks()) + .andReturn(ImmutableList.of(TASK_RUNNER_WORK_ITEM)) + .anyTimes(); + Capture> capturedArguments = EasyMock.newCapture(); + STORAGE_CONNECTOR.deleteFiles(EasyMock.capture(capturedArguments)); + EasyMock.expectLastCall().once(); + EasyMock.replay(TASK_MASTER, TASK_RUNNER, TASK_RUNNER_WORK_ITEM, STORAGE_CONNECTOR); + + durableStorageCleaner.run(); + + Assert.assertEquals(Sets.newHashSet(STRAY_DIR, intermediateFilesPath), capturedArguments.getValue()); + } + @Test public void testGetSchedule() { diff --git a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java index 8aea264bc3b..1585b865157 100644 --- a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java @@ -20,11 +20,14 @@ package org.apache.druid.frame.util; import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableList; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nullable; import java.util.Iterator; +import java.util.List; +import java.util.Set; /** * Helper class that fetches the directory and file names corresponding to file location @@ -32,7 +35,8 @@ import java.util.Iterator; public class DurableStorageUtils { public static final String SUCCESS_MARKER_FILENAME = "__success"; - public static final Splitter SPLITTER = Splitter.on("/").limit(2); + public static final Splitter SPLITTER = Splitter.on("/").limit(3); + public static final String QUERY_RESULTS_DIR = "query-results"; public static String getControllerDirectory(final String controllerTaskId) { @@ -127,7 +131,7 @@ public class DurableStorageUtils } /** - * Tries to parse out the controller taskID from the input path. + * Tries to parse out the most top level directory from the path. Returns null if there is no such directory. *

* For eg: *
@@ -138,7 +142,7 @@ public class DurableStorageUtils * */ @Nullable - public static String getControllerTaskIdWithPrefixFromPath(String path) + public static String getNextDirNameWithPrefixFromPath(String path) { if (path == null) { return null; @@ -150,4 +154,35 @@ public class DurableStorageUtils return null; } } + + /** + * Tries to parse out the controller taskID from the query results path, and checks if the taskID is present in the + * set of known tasks. + * Returns true if the set contains the taskId. + * Returns false if taskId could not be parsed or if the set does not contain the taskId. + *

+ * For eg: + *
+ *

    + *
  • for path controller_query_id/task/123 the function will return false
  • + *
  • for path query-result/controller_query_id/results.json, the function will return true
  • if the controller_query_id is in known tasks + *
  • for path query-result/controller_query_id/results.json, the function will return false
  • if the controller_query_id is not in known tasks + *
  • for path null, the function will return false
  • + *
+ */ + public static boolean isQueryResultFileActive(String path, Set knownTasks) + { + if (path == null) { + return false; + } + Iterator elementsIterator = SPLITTER.split(path).iterator(); + List elements = ImmutableList.copyOf(elementsIterator); + if (elements.size() < 2) { + return false; + } + if (!DurableStorageUtils.QUERY_RESULTS_DIR.equals(elements.get(0))) { + return false; + } + return knownTasks.contains(elements.get(1)); + } } diff --git a/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java b/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java index 8e320381078..71b16633e92 100644 --- a/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/frame/util/DurableStorageUtilsTest.java @@ -19,6 +19,7 @@ package org.apache.druid.frame.util; +import com.google.common.collect.ImmutableSet; import org.junit.Assert; import org.junit.Test; @@ -26,13 +27,39 @@ public class DurableStorageUtilsTest { @Test - public void getControllerTaskIdWithPrefixFromPath() + public void getNextDirNameWithPrefixFromPath() { - Assert.assertEquals("", DurableStorageUtils.getControllerTaskIdWithPrefixFromPath("/123/123")); - Assert.assertEquals("123", DurableStorageUtils.getControllerTaskIdWithPrefixFromPath("123")); + Assert.assertEquals("", DurableStorageUtils.getNextDirNameWithPrefixFromPath("/123/123")); + Assert.assertEquals("123", DurableStorageUtils.getNextDirNameWithPrefixFromPath("123")); Assert.assertEquals("controller_query_123", - DurableStorageUtils.getControllerTaskIdWithPrefixFromPath("controller_query_123/123")); - Assert.assertEquals("", DurableStorageUtils.getControllerTaskIdWithPrefixFromPath("")); - Assert.assertNull(DurableStorageUtils.getControllerTaskIdWithPrefixFromPath(null)); + DurableStorageUtils.getNextDirNameWithPrefixFromPath("controller_query_123/123")); + Assert.assertEquals("", DurableStorageUtils.getNextDirNameWithPrefixFromPath("")); + Assert.assertNull(DurableStorageUtils.getNextDirNameWithPrefixFromPath(null)); + } + + @Test + public void isQueryResultFileActive() + { + + Assert.assertTrue(DurableStorageUtils.isQueryResultFileActive( + DurableStorageUtils.QUERY_RESULTS_DIR + "/123/result", + ImmutableSet.of("123") + )); + Assert.assertFalse(DurableStorageUtils.isQueryResultFileActive( + DurableStorageUtils.QUERY_RESULTS_DIR + "/123/result", + ImmutableSet.of("") + )); + Assert.assertFalse(DurableStorageUtils.isQueryResultFileActive( + DurableStorageUtils.QUERY_RESULTS_DIR + "/", + ImmutableSet.of("123") + )); + Assert.assertFalse(DurableStorageUtils.isQueryResultFileActive( + null, + ImmutableSet.of("123") + )); + Assert.assertFalse(DurableStorageUtils.isQueryResultFileActive( + DurableStorageUtils.QUERY_RESULTS_DIR, + ImmutableSet.of("123") + )); } } From fd20bbd30ecee969fd3f38cbe1d45de7d763f531 Mon Sep 17 00:00:00 2001 From: imply-cheddar <86940447+imply-cheddar@users.noreply.github.com> Date: Wed, 28 Jun 2023 13:49:27 +0900 Subject: [PATCH 41/74] Fix another infinite loop and remove Mockito usage (#14493) * Fix another infinite loop and remove Mockito usage The ConfigManager objects were `started()` without ever being stopped. This scheduled a poll call that never-ended, to make matters worse, the poll interval was set to 0 ms, making an infinite poll with 0 sleep, i.e. an infinite loop. Also introduce test classes and remove usage of mocks * Checkstyle --- .../apache/druid/audit/TestAuditManager.java | 71 +++++++ .../common/config/ConfigManagerTest.java | 182 +++++++++++------- .../config/TestConfigManagerConfig.java | 40 ++++ .../TestMetadataStorageConnector.java | 92 +++++++++ .../TestMetadataStorageTablesConfig.java | 43 +++++ 5 files changed, 361 insertions(+), 67 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/audit/TestAuditManager.java create mode 100644 processing/src/test/java/org/apache/druid/common/config/TestConfigManagerConfig.java create mode 100644 processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java create mode 100644 processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java diff --git a/processing/src/test/java/org/apache/druid/audit/TestAuditManager.java b/processing/src/test/java/org/apache/druid/audit/TestAuditManager.java new file mode 100644 index 00000000000..fd34330a68b --- /dev/null +++ b/processing/src/test/java/org/apache/druid/audit/TestAuditManager.java @@ -0,0 +1,71 @@ +/* + * 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.druid.audit; + +import org.apache.druid.common.config.ConfigSerde; +import org.joda.time.Interval; +import org.skife.jdbi.v2.Handle; + +import java.util.List; + +public class TestAuditManager implements AuditManager +{ + @Override + public void doAudit(String key, String type, AuditInfo auditInfo, T payload, ConfigSerde configSerde) + { + throw new UnsupportedOperationException(); + } + + @Override + public void doAudit(AuditEntry auditEntry, Handle handler) + { + throw new UnsupportedOperationException(); + } + + @Override + public List fetchAuditHistory(String key, String type, Interval interval) + { + throw new UnsupportedOperationException(); + } + + @Override + public List fetchAuditHistory(String type, Interval interval) + { + throw new UnsupportedOperationException(); + } + + @Override + public List fetchAuditHistory(String key, String type, int limit) + { + throw new UnsupportedOperationException(); + } + + @Override + public List fetchAuditHistory(String type, int limit) + { + throw new UnsupportedOperationException(); + } + + @Override + public int removeAuditLogsOlderThan(long timestamp) + { + throw new UnsupportedOperationException(); + } +} diff --git a/processing/src/test/java/org/apache/druid/common/config/ConfigManagerTest.java b/processing/src/test/java/org/apache/druid/common/config/ConfigManagerTest.java index 9705939991f..2f58147e77c 100644 --- a/processing/src/test/java/org/apache/druid/common/config/ConfigManagerTest.java +++ b/processing/src/test/java/org/apache/druid/common/config/ConfigManagerTest.java @@ -25,26 +25,19 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; import org.apache.druid.audit.AuditManager; +import org.apache.druid.audit.TestAuditManager; import org.apache.druid.metadata.MetadataCASUpdate; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; -import org.joda.time.Period; +import org.apache.druid.metadata.TestMetadataStorageConnector; +import org.apache.druid.metadata.TestMetadataStorageTablesConfig; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatchers; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.MockitoJUnitRunner; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.when; - -@RunWith(MockitoJUnitRunner.class) +@SuppressWarnings("ALL") public class ConfigManagerTest { private static final String CONFIG_KEY = "configX"; @@ -52,33 +45,42 @@ public class ConfigManagerTest private static final byte[] OLD_CONFIG = {1, 2, 3}; private static final TestConfig NEW_CONFIG = new TestConfig("2", "y", 2); - @Mock - private MetadataStorageConnector mockDbConnector; - - @Mock - private MetadataStorageTablesConfig mockMetadataStorageTablesConfig; - - @Mock + private MetadataStorageConnector dbConnector; + private MetadataStorageTablesConfig metadataStorageTablesConfig; private AuditManager mockAuditManager; - - @Mock - private ConfigManagerConfig mockConfigManagerConfig; + private TestConfigManagerConfig configManagerConfig; private ConfigSerde configConfigSerdeFromClass; private ConfigManager configManager; private JacksonConfigManager jacksonConfigManager; - @Before public void setup() { - when(mockMetadataStorageTablesConfig.getConfigTable()).thenReturn(TABLE_NAME); - when(mockConfigManagerConfig.getPollDuration()).thenReturn(new Period()); - configManager = new ConfigManager(mockDbConnector, Suppliers.ofInstance(mockMetadataStorageTablesConfig), Suppliers.ofInstance(mockConfigManagerConfig)); + setup(new TestMetadataStorageConnector()); + } + + public void setup(TestMetadataStorageConnector dbConnector) + { + this.dbConnector = dbConnector; + metadataStorageTablesConfig = new TestMetadataStorageTablesConfig() + { + @Override + public String getConfigTable() + { + return TABLE_NAME; + } + }; + configManagerConfig = new TestConfigManagerConfig(); + configManager = new ConfigManager( + this.dbConnector, + Suppliers.ofInstance(metadataStorageTablesConfig), + Suppliers.ofInstance(configManagerConfig) + ); jacksonConfigManager = new JacksonConfigManager( configManager, new ObjectMapper(), new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL), - mockAuditManager + new TestAuditManager() ); configConfigSerdeFromClass = jacksonConfigManager.create(TestConfig.class, null); } @@ -86,6 +88,7 @@ public class ConfigManagerTest @Test public void testSetNewObjectIsNull() { + setup(); ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, null); Assert.assertFalse(setResult.isOk()); Assert.assertFalse(setResult.isRetryable()); @@ -95,6 +98,7 @@ public class ConfigManagerTest @Test public void testSetConfigManagerNotStarted() { + setup(); ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, NEW_CONFIG); Assert.assertFalse(setResult.isOk()); Assert.assertFalse(setResult.isRetryable()); @@ -104,56 +108,100 @@ public class ConfigManagerTest @Test public void testSetOldObjectNullShouldInsertWithoutSwap() { - configManager.start(); - ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, null, NEW_CONFIG); - Assert.assertTrue(setResult.isOk()); - Mockito.verify(mockDbConnector).insertOrUpdate( - ArgumentMatchers.eq(TABLE_NAME), - ArgumentMatchers.anyString(), - ArgumentMatchers.anyString(), - ArgumentMatchers.eq(CONFIG_KEY), - ArgumentMatchers.any(byte[].class) - ); - Mockito.verifyNoMoreInteractions(mockDbConnector); + final AtomicBoolean called = new AtomicBoolean(); + setup(new TestMetadataStorageConnector() + { + + @Override + public Void insertOrUpdate(String tableName, String keyColumn, String valueColumn, String key, byte[] value) + { + Assert.assertFalse(called.getAndSet(true)); + Assert.assertEquals(TABLE_NAME, tableName); + Assert.assertEquals(CONFIG_KEY, key); + return null; + } + }); + + try { + configManager.start(); + ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, null, NEW_CONFIG); + Assert.assertTrue(setResult.isOk()); + Assert.assertTrue(called.get()); + } + finally { + configManager.stop(); + } } @Test public void testSetOldObjectNotNullShouldSwap() { - when(mockConfigManagerConfig.isEnableCompareAndSwap()).thenReturn(true); - when(mockDbConnector.compareAndSwap(any(List.class))).thenReturn(true); - final ArgumentCaptor> updateCaptor = ArgumentCaptor.forClass(List.class); - configManager.start(); - ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, OLD_CONFIG, NEW_CONFIG); - Assert.assertTrue(setResult.isOk()); - Mockito.verify(mockDbConnector).compareAndSwap( - updateCaptor.capture() - ); - Mockito.verifyNoMoreInteractions(mockDbConnector); - Assert.assertEquals(1, updateCaptor.getValue().size()); - Assert.assertEquals(TABLE_NAME, updateCaptor.getValue().get(0).getTableName()); - Assert.assertEquals(MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, updateCaptor.getValue().get(0).getKeyColumn()); - Assert.assertEquals(MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, updateCaptor.getValue().get(0).getValueColumn()); - Assert.assertEquals(CONFIG_KEY, updateCaptor.getValue().get(0).getKey()); - Assert.assertArrayEquals(OLD_CONFIG, updateCaptor.getValue().get(0).getOldValue()); - Assert.assertArrayEquals(configConfigSerdeFromClass.serialize(NEW_CONFIG), updateCaptor.getValue().get(0).getNewValue()); + final AtomicBoolean called = new AtomicBoolean(); + setup(new TestMetadataStorageConnector() + { + @Override + public boolean compareAndSwap(List updates) + { + Assert.assertFalse(called.getAndSet(true)); + Assert.assertEquals(1, updates.size()); + final MetadataCASUpdate singularUpdate = updates.get(0); + Assert.assertEquals(TABLE_NAME, singularUpdate.getTableName()); + Assert.assertEquals(MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, singularUpdate.getKeyColumn()); + Assert.assertEquals(MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, singularUpdate.getValueColumn()); + Assert.assertEquals(CONFIG_KEY, singularUpdate.getKey()); + Assert.assertArrayEquals(OLD_CONFIG, singularUpdate.getOldValue()); + Assert.assertArrayEquals(configConfigSerdeFromClass.serialize(NEW_CONFIG), singularUpdate.getNewValue()); + return true; + } + }); + try { + configManager.start(); + ConfigManager.SetResult setResult = configManager.set( + CONFIG_KEY, + configConfigSerdeFromClass, + OLD_CONFIG, + NEW_CONFIG + ); + Assert.assertTrue(setResult.isOk()); + Assert.assertTrue(called.get()); + } + finally { + configManager.stop(); + } } @Test public void testSetOldObjectNotNullButCompareAndSwapDisabledShouldInsertWithoutSwap() { - when(mockConfigManagerConfig.isEnableCompareAndSwap()).thenReturn(false); - configManager.start(); - ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, OLD_CONFIG, NEW_CONFIG); - Assert.assertTrue(setResult.isOk()); - Mockito.verify(mockDbConnector).insertOrUpdate( - ArgumentMatchers.eq(TABLE_NAME), - ArgumentMatchers.anyString(), - ArgumentMatchers.anyString(), - ArgumentMatchers.eq(CONFIG_KEY), - ArgumentMatchers.any(byte[].class) - ); - Mockito.verifyNoMoreInteractions(mockDbConnector); + final AtomicBoolean called = new AtomicBoolean(); + + setup(new TestMetadataStorageConnector() + { + @Override + public Void insertOrUpdate(String tableName, String keyColumn, String valueColumn, String key, byte[] value) + { + Assert.assertFalse(called.getAndSet(true)); + Assert.assertEquals(TABLE_NAME, tableName); + Assert.assertEquals(CONFIG_KEY, key); + return null; + } + }); + configManagerConfig.enableCompareAndSwap = false; + + try { + configManager.start(); + ConfigManager.SetResult setResult = configManager.set( + CONFIG_KEY, + configConfigSerdeFromClass, + OLD_CONFIG, + NEW_CONFIG + ); + Assert.assertTrue(setResult.isOk()); + Assert.assertTrue(called.get()); + } + finally { + configManager.stop(); + } } static class TestConfig diff --git a/processing/src/test/java/org/apache/druid/common/config/TestConfigManagerConfig.java b/processing/src/test/java/org/apache/druid/common/config/TestConfigManagerConfig.java new file mode 100644 index 00000000000..69ff82ca2a0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/common/config/TestConfigManagerConfig.java @@ -0,0 +1,40 @@ +/* + * 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.druid.common.config; + +import org.joda.time.Period; + +public class TestConfigManagerConfig extends ConfigManagerConfig +{ + public Period period = null; + public boolean enableCompareAndSwap = true; + + @Override + public Period getPollDuration() + { + return period == null ? super.getPollDuration() : period; + } + + @Override + public boolean isEnableCompareAndSwap() + { + return enableCompareAndSwap; + } +} diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java new file mode 100644 index 00000000000..028a9d5cc08 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java @@ -0,0 +1,92 @@ +/* + * 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.druid.metadata; + +import javax.annotation.Nullable; + +public class TestMetadataStorageConnector implements MetadataStorageConnector +{ + @Override + public Void insertOrUpdate(String tableName, String keyColumn, String valueColumn, String key, byte[] value) + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public byte[] lookup(String tableName, String keyColumn, String valueColumn, String key) + { + throw new UnsupportedOperationException(); + } + + @Override + public void createDataSourceTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createPendingSegmentsTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createSegmentTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createRulesTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createConfigTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createTaskTables() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createAuditTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createSupervisorsTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteAllRecords(String tableName) + { + throw new UnsupportedOperationException(); + } +} diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java new file mode 100644 index 00000000000..693b36ce24e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java @@ -0,0 +1,43 @@ +/* + * 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.druid.metadata; + +/** + * A config object for tests, use by overriding the specific getter methods and returning what you want + */ +public class TestMetadataStorageTablesConfig extends MetadataStorageTablesConfig +{ + public TestMetadataStorageTablesConfig() + { + super( + "test", + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + } +} From baa64e6d8a71b1d3816dea6144ca4322d0432e53 Mon Sep 17 00:00:00 2001 From: Tejaswini Bandlamudi <96047043+tejaswini-imply@users.noreply.github.com> Date: Wed, 28 Jun 2023 15:03:10 +0530 Subject: [PATCH 42/74] update hadoop version to 3.3.6 (#14489) --- licenses.yaml | 4 ++-- owasp-dependency-check-suppressions.xml | 2 +- pom.xml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/licenses.yaml b/licenses.yaml index fe46b80a12b..8efba06fe84 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -2924,7 +2924,7 @@ name: Apache Hadoop license_category: binary module: hadoop-client license_name: Apache License version 2.0 -version: 3.3.5 +version: 3.3.6 libraries: - org.apache.hadoop: hadoop-auth - org.apache.hadoop: hadoop-common @@ -3826,7 +3826,7 @@ name: Hadoop Client API license_category: binary module: extensions/druid-hdfs-storage license_name: Apache License version 2.0 -version: 3.3.5 +version: 3.3.6 libraries: - org.apache.hadoop: hadoop-client-api diff --git a/owasp-dependency-check-suppressions.xml b/owasp-dependency-check-suppressions.xml index e4c2ddf73c0..886750cad0e 100644 --- a/owasp-dependency-check-suppressions.xml +++ b/owasp-dependency-check-suppressions.xml @@ -810,7 +810,7 @@ CVE-2022-26612 diff --git a/pom.xml b/pom.xml index f193957ee6a..6d0d1337498 100644 --- a/pom.xml +++ b/pom.xml @@ -111,7 +111,7 @@ 1.7.36 5.13.0 5.13.0 - 3.3.5 + 3.3.6 4.3.1 1.12.317 2.8.0 From cb3a9d2b5778c568f8a6fad096b95bfd1f36bd21 Mon Sep 17 00:00:00 2001 From: Karan Kumar Date: Wed, 28 Jun 2023 17:51:58 +0530 Subject: [PATCH 43/74] Adding Interactive API's for MSQ engine (#14416) This PR aims to expose a new API called "@path("/druid/v2/sql/statements/")" which takes the same payload as the current "/druid/v2/sql" endpoint and allows users to fetch results in an async manner. --- .../apache/druid/msq/guice/SqlTaskModule.java | 9 +- .../druid/msq/indexing/MSQControllerTask.java | 18 +- .../error/InsertCannotBeEmptyFault.java | 2 +- .../druid/msq/sql/MSQTaskQueryMaker.java | 9 +- .../druid/msq/sql/MSQTaskSqlEngine.java | 4 +- .../druid/msq/sql/SqlStatementState.java | 42 + .../apache/druid/msq/sql/SqlTaskStatus.java | 1 + .../msq/sql/entity/ColumnNameAndTypes.java | 102 ++ .../msq/sql/entity/ResultSetInformation.java | 142 +++ .../msq/sql/entity/SqlStatementResult.java | 185 ++++ .../sql/resources/SqlStatementResource.java | 670 +++++++++++++ .../sql/{ => resources}/SqlTaskResource.java | 4 +- .../msq/util/SqlStatementResourceHelper.java | 285 ++++++ .../msq/indexing/MSQControllerTaskTest.java | 1 + .../indexing/report/MSQTaskReportTest.java | 2 +- .../sql/SqlMsqStatementResourcePostTest.java | 256 +++++ .../msq/sql/SqlStatementResourceTest.java | 947 ++++++++++++++++++ .../sql/entity/ColumnNameAndTypesTest.java | 53 + .../sql/entity/ResultSetInformationTest.java | 58 ++ .../sql/entity/SqlStatementResultTest.java | 95 ++ .../apache/druid/msq/test/MSQTestBase.java | 24 +- .../test/MSQTestOverlordServiceClient.java | 83 +- .../druid/indexing/common/task/IndexTask.java | 2 +- .../apache/druid/error/DruidException.java | 41 +- .../org/apache/druid/query/ExecutionMode.java | 35 + .../org/apache/druid/query/QueryContexts.java | 17 +- .../apache/druid/query/QueryContextsTest.java | 22 + .../druid/rpc/indexing/OverlordClient.java | 3 + .../rpc/indexing/OverlordClientImpl.java | 17 + .../client/indexing/NoopOverlordClient.java | 6 + .../rpc/indexing/OverlordClientImplTest.java | 58 ++ .../org/apache/druid/sql/DirectStatement.java | 4 +- .../druid/sql/calcite/util/CalciteTests.java | 34 + .../druid/sql/http/SqlResourceTest.java | 2 +- 34 files changed, 3170 insertions(+), 63 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypes.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/{ => resources}/SqlTaskResource.java (98%) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypesTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java create mode 100644 processing/src/main/java/org/apache/druid/query/ExecutionMode.java create mode 100644 server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java index d8f8b06f9b9..bcd8abfbb87 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java @@ -21,12 +21,16 @@ package org.apache.druid.msq.guice; import com.fasterxml.jackson.databind.Module; import com.google.inject.Binder; +import org.apache.druid.client.indexing.HttpIndexingServiceClient; +import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.annotations.LoadScope; import org.apache.druid.initialization.DruidModule; -import org.apache.druid.msq.sql.SqlTaskResource; +import org.apache.druid.msq.sql.resources.SqlStatementResource; +import org.apache.druid.msq.sql.resources.SqlTaskResource; import java.util.Collections; import java.util.List; @@ -43,6 +47,9 @@ public class SqlTaskModule implements DruidModule // Force eager initialization. LifecycleModule.register(binder, SqlTaskResource.class); Jerseys.addResource(binder, SqlTaskResource.class); + LifecycleModule.register(binder, SqlStatementResource.class); + binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); + Jerseys.addResource(binder, SqlStatementResource.class); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index 3265f2640af..02280525738 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -30,6 +30,7 @@ import com.google.common.collect.ImmutableSet; import com.google.inject.Injector; import com.google.inject.Key; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; @@ -49,6 +50,7 @@ import org.apache.druid.msq.exec.MSQTasks; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.calcite.run.SqlResults; import org.joda.time.Interval; @@ -60,7 +62,7 @@ import java.util.Map; import java.util.Set; @JsonTypeName(MSQControllerTask.TYPE) -public class MSQControllerTask extends AbstractTask +public class MSQControllerTask extends AbstractTask implements ClientTaskQuery { public static final String TYPE = "query_controller"; public static final String DUMMY_DATASOURCE_FOR_SELECT = "__query_select"; @@ -91,6 +93,9 @@ public class MSQControllerTask extends AbstractTask @Nullable private final List sqlTypeNames; + @Nullable + private final List nativeTypeNames; + // Using an Injector directly because tasks do not have a way to provide their own Guice modules. @JacksonInject private Injector injector; @@ -105,6 +110,7 @@ public class MSQControllerTask extends AbstractTask @JsonProperty("sqlQueryContext") @Nullable Map sqlQueryContext, @JsonProperty("sqlResultsContext") @Nullable SqlResults.Context sqlResultsContext, @JsonProperty("sqlTypeNames") @Nullable List sqlTypeNames, + @JsonProperty("nativeTypeNames") @Nullable List nativeTypeNames, @JsonProperty("context") @Nullable Map context ) { @@ -121,6 +127,7 @@ public class MSQControllerTask extends AbstractTask this.sqlQueryContext = sqlQueryContext; this.sqlResultsContext = sqlResultsContext; this.sqlTypeNames = sqlTypeNames; + this.nativeTypeNames = nativeTypeNames; addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } @@ -154,6 +161,15 @@ public class MSQControllerTask extends AbstractTask return sqlTypeNames; } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getNativeTypeNames() + { + return nativeTypeNames; + } + @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java index 4285ace4180..7948e813615 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java @@ -29,7 +29,7 @@ import java.util.Objects; @JsonTypeName(InsertCannotBeEmptyFault.CODE) public class InsertCannotBeEmptyFault extends BaseMSQFault { - static final String CODE = "InsertCannotBeEmpty"; + public static final String CODE = "InsertCannotBeEmpty"; private final String dataSource; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index cde4906b193..c0de08a809f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -78,6 +78,8 @@ public class MSQTaskQueryMaker implements QueryMaker private static final String DESTINATION_DATASOURCE = "dataSource"; private static final String DESTINATION_REPORT = "taskReport"; + public static final String USER_KEY = "__user"; + private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; private final String targetDataSource; @@ -116,6 +118,9 @@ public class MSQTaskQueryMaker implements QueryMaker // Native query context: sqlQueryContext plus things that we add prior to creating a controller task. final Map nativeQueryContext = new HashMap<>(sqlQueryContext.asMap()); + // adding user + nativeQueryContext.put(USER_KEY, plannerContext.getAuthenticationResult().getIdentity()); + final String msqMode = MultiStageQueryContext.getMSQMode(sqlQueryContext); if (msqMode != null) { MSQMode.populateDefaultQueryContext(msqMode, nativeQueryContext); @@ -174,6 +179,7 @@ public class MSQTaskQueryMaker implements QueryMaker finalizeAggregations ? null /* Not needed */ : buildAggregationIntermediateTypeMap(druidQuery); final List sqlTypeNames = new ArrayList<>(); + final List columnTypeList = new ArrayList<>(); final List columnMappings = QueryUtils.buildColumnMappings(fieldMapping, druidQuery); for (final Pair entry : fieldMapping) { @@ -187,8 +193,8 @@ public class MSQTaskQueryMaker implements QueryMaker } else { sqlTypeName = druidQuery.getOutputRowType().getFieldList().get(entry.getKey()).getType().getSqlTypeName(); } - sqlTypeNames.add(sqlTypeName); + columnTypeList.add(druidQuery.getOutputRowSignature().getColumnType(queryColumn).orElse(ColumnType.STRING)); } final MSQDestination destination; @@ -248,6 +254,7 @@ public class MSQTaskQueryMaker implements QueryMaker plannerContext.queryContextMap(), SqlResults.Context.fromPlannerContext(plannerContext), sqlTypeNames, + columnTypeList, null ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java index 7d27103e0b9..c5fe182ea96 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java @@ -296,7 +296,9 @@ public class MSQTaskSqlEngine implements SqlEngine private static RelDataType getMSQStructType(RelDataTypeFactory typeFactory) { return typeFactory.createStructType( - ImmutableList.of(Calcites.createSqlType(typeFactory, SqlTypeName.VARCHAR)), + ImmutableList.of( + Calcites.createSqlType(typeFactory, SqlTypeName.VARCHAR) + ), TASK_STRUCT_FIELD_NAMES ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java new file mode 100644 index 00000000000..e7f7da95d8e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java @@ -0,0 +1,42 @@ +/* + * 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.druid.msq.sql; + +import org.apache.druid.sql.http.SqlQuery; + +import javax.servlet.http.HttpServletRequest; + +/** + * Represents the status of the sql statements issues via + * {@link org.apache.druid.msq.sql.resources.SqlStatementResource#doPost(SqlQuery, HttpServletRequest)} and returned in + * {@link org.apache.druid.msq.sql.entity.SqlStatementResult} + */ +public enum SqlStatementState +{ + // The statement is accepted but not yes assigned any worker. In MSQ engine, the statement is in ACCEPTED state + // till the overlord assigns a TaskLocation to the controller task. + ACCEPTED, + // The statement is running. + RUNNING, + // The statement is successful. + SUCCESS, + // The statement failed. + FAILED +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java index 9fa14ee9944..9c92a2589f2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.TaskState; +import org.apache.druid.msq.sql.resources.SqlTaskResource; import javax.annotation.Nullable; import java.util.Objects; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypes.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypes.java new file mode 100644 index 00000000000..3f7188c5d93 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypes.java @@ -0,0 +1,102 @@ +/* + * 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.druid.msq.sql.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * The column name and its sql {@link org.apache.calcite.sql.type.SqlTypeName} and its native {@link org.apache.druid.segment.column.ColumnType} + */ + +public class ColumnNameAndTypes +{ + + private final String colName; + private final String sqlTypeName; + + private final String nativeTypeName; + + @JsonCreator + public ColumnNameAndTypes( + @JsonProperty("name") String colName, + @JsonProperty("type") String sqlTypeName, + @JsonProperty("nativeType") String nativeTypeName + ) + { + + this.colName = colName; + this.sqlTypeName = sqlTypeName; + this.nativeTypeName = nativeTypeName; + } + + @JsonProperty("name") + public String getColName() + { + return colName; + } + + @JsonProperty("type") + public String getSqlTypeName() + { + return sqlTypeName; + } + + @JsonProperty("nativeType") + public String getNativeTypeName() + { + return nativeTypeName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ColumnNameAndTypes that = (ColumnNameAndTypes) o; + return Objects.equals(colName, that.colName) + && Objects.equals(sqlTypeName, that.sqlTypeName) + && Objects.equals(nativeTypeName, that.nativeTypeName); + } + + @Override + public int hashCode() + { + return Objects.hash(colName, sqlTypeName, nativeTypeName); + } + + @Override + public String toString() + { + return "ColumnNameAndTypes{" + + "colName='" + colName + '\'' + + ", sqlTypeName='" + sqlTypeName + '\'' + + ", nativeTypeName='" + nativeTypeName + '\'' + + '}'; + } +} + + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java new file mode 100644 index 00000000000..43201fdac6e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java @@ -0,0 +1,142 @@ +/* + * 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.druid.msq.sql.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.sql.http.ResultFormat; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +public class ResultSetInformation +{ + + @Nullable + private final Long numRows; + @Nullable + private final Long sizeInBytes; + + @Nullable + private final ResultFormat resultFormat; + + @Nullable + private final List records; + + @Nullable + private final String dataSource; + + @JsonCreator + public ResultSetInformation( + @JsonProperty("resultFormat") @Nullable ResultFormat resultFormat, + @JsonProperty("numRows") @Nullable Long numRows, + @JsonProperty("sizeInBytes") @Nullable Long sizeInBytes, + @JsonProperty("dataSource") @Nullable String dataSource, + @JsonProperty("sampleRecords") @Nullable + List records + ) + { + this.numRows = numRows; + this.sizeInBytes = sizeInBytes; + this.resultFormat = resultFormat; + this.dataSource = dataSource; + this.records = records; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Long getNumRows() + { + return numRows; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Long getSizeInBytes() + { + return sizeInBytes; + } + + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public ResultFormat getResultFormat() + { + return resultFormat; + } + + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getDataSource() + { + return dataSource; + } + + @Nullable + @JsonProperty("sampleRecords") + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getRecords() + { + return records; + } + + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ResultSetInformation that = (ResultSetInformation) o; + return Objects.equals(numRows, that.numRows) + && Objects.equals(sizeInBytes, that.sizeInBytes) + && resultFormat == that.resultFormat + && Objects.equals(records, that.records) + && Objects.equals(dataSource, that.dataSource); + } + + @Override + public int hashCode() + { + return Objects.hash(numRows, sizeInBytes, resultFormat, records, dataSource); + } + + @Override + public String toString() + { + return "ResultSetInformation{" + + "totalRows=" + numRows + + ", totalSize=" + sizeInBytes + + ", resultFormat=" + resultFormat + + ", records=" + records + + ", dataSource='" + dataSource + '\'' + + '}'; + } + +} + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java new file mode 100644 index 00000000000..de66550a587 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -0,0 +1,185 @@ +/* + * 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.druid.msq.sql.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.ErrorResponse; +import org.apache.druid.msq.sql.SqlStatementState; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +public class SqlStatementResult +{ + + private final String queryId; + + private final SqlStatementState state; + + private final DateTime createdAt; + + @Nullable + private final List sqlRowSignature; + + @Nullable + private final Long durationMs; + + @Nullable + private final ResultSetInformation resultSetInformation; + + @Nullable + private final ErrorResponse errorResponse; + + + @JsonCreator + public SqlStatementResult( + @JsonProperty("queryId") + String queryId, + @JsonProperty("state") + SqlStatementState state, + @JsonProperty("createdAt") + DateTime createdAt, + @Nullable @JsonProperty("schema") + List sqlRowSignature, + @Nullable @JsonProperty("durationMs") + Long durationMs, + @Nullable @JsonProperty("result") + ResultSetInformation resultSetInformation, + @Nullable @JsonProperty("errorDetails") + ErrorResponse errorResponse + + ) + { + this.queryId = queryId; + this.state = state; + this.createdAt = createdAt; + this.sqlRowSignature = sqlRowSignature; + this.durationMs = durationMs; + this.resultSetInformation = resultSetInformation; + this.errorResponse = errorResponse; + } + + @JsonProperty + public String getQueryId() + { + return queryId; + } + + @JsonProperty + public SqlStatementState getState() + { + return state; + } + + @JsonProperty + public DateTime getCreatedAt() + { + return createdAt; + } + + @JsonProperty("schema") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getSqlRowSignature() + { + return sqlRowSignature; + } + + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public Long getDurationMs() + { + return durationMs; + } + + @JsonProperty("result") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public ResultSetInformation getResultSetInformation() + { + return resultSetInformation; + } + + @JsonProperty("errorDetails") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public ErrorResponse getErrorResponse() + { + return errorResponse; + } + + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SqlStatementResult that = (SqlStatementResult) o; + return Objects.equals(queryId, that.queryId) && state == that.state && Objects.equals( + createdAt, + that.createdAt + ) && Objects.equals(sqlRowSignature, that.sqlRowSignature) && Objects.equals( + durationMs, + that.durationMs + ) && Objects.equals(resultSetInformation, that.resultSetInformation) && Objects.equals( + errorResponse == null ? null : errorResponse.getAsMap(), + that.errorResponse == null ? null : that.errorResponse.getAsMap() + ); + } + + @Override + public int hashCode() + { + return Objects.hash( + queryId, + state, + createdAt, + sqlRowSignature, + durationMs, + resultSetInformation, + errorResponse == null ? null : errorResponse.getAsMap() + ); + } + + @Override + public String toString() + { + return "SqlStatementResult{" + + "queryId='" + queryId + '\'' + + ", state=" + state + + ", createdAt=" + createdAt + + ", sqlRowSignature=" + sqlRowSignature + + ", durationInMs=" + durationMs + + ", resultSetInformation=" + resultSetInformation + + ", errorResponse=" + (errorResponse == null + ? "{}" + : errorResponse.getAsMap().toString()) + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java new file mode 100644 index 00000000000..6ce5c780052 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -0,0 +1,670 @@ +/* + * 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.druid.msq.sql.resources; + + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.CountingOutputStream; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Inject; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.ErrorResponse; +import org.apache.druid.error.QueryExceptionCompat; +import org.apache.druid.guice.annotations.MSQ; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.sql.MSQTaskQueryMaker; +import org.apache.druid.msq.sql.MSQTaskSqlEngine; +import org.apache.druid.msq.sql.SqlStatementState; +import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; +import org.apache.druid.msq.sql.entity.ResultSetInformation; +import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.msq.util.SqlStatementResourceHelper; +import org.apache.druid.query.ExecutionMode; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryException; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.server.QueryResponse; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.sql.DirectStatement; +import org.apache.druid.sql.HttpStatement; +import org.apache.druid.sql.SqlRowTransformer; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.sql.http.SqlQuery; +import org.apache.druid.sql.http.SqlResource; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.StreamingOutput; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; + + +@Path("/druid/v2/sql/statements/") +public class SqlStatementResource +{ + + private static final Logger log = new Logger(SqlStatementResource.class); + private final SqlStatementFactory msqSqlStatementFactory; + private final AuthorizerMapper authorizerMapper; + private final ObjectMapper jsonMapper; + private final OverlordClient overlordClient; + + + @Inject + public SqlStatementResource( + final @MSQ SqlStatementFactory msqSqlStatementFactory, + final AuthorizerMapper authorizerMapper, + final ObjectMapper jsonMapper, + final OverlordClient overlordClient + ) + { + this.msqSqlStatementFactory = msqSqlStatementFactory; + this.authorizerMapper = authorizerMapper; + this.jsonMapper = jsonMapper; + this.overlordClient = overlordClient; + } + + + @GET + @Path("/enabled") + @Produces(MediaType.APPLICATION_JSON) + public Response isEnabled(@Context final HttpServletRequest request) + { + // All authenticated users are authorized for this API: check an empty resource list. + final Access authResult = AuthorizationUtils.authorizeAllResourceActions( + request, + Collections.emptyList(), + authorizerMapper + ); + + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + + return Response.ok(ImmutableMap.of("enabled", true)).build(); + } + + @POST + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response doPost(final SqlQuery sqlQuery, @Context final HttpServletRequest req) + { + final HttpStatement stmt = msqSqlStatementFactory.httpStatement(sqlQuery, req); + final String sqlQueryId = stmt.sqlQueryId(); + final String currThreadName = Thread.currentThread().getName(); + try { + ExecutionMode executionMode = QueryContexts.getAsEnum( + QueryContexts.CTX_EXECUTION_MODE, + sqlQuery.getContext().get(QueryContexts.CTX_EXECUTION_MODE), + ExecutionMode.class + ); + if (ExecutionMode.ASYNC != executionMode) { + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + StringUtils.format( + "The statement sql api only supports sync mode[%s]. Please set context parameter [%s=%s] in the context payload", + ExecutionMode.ASYNC, + QueryContexts.CTX_EXECUTION_MODE, + ExecutionMode.ASYNC + ) + ) + ); + } + + + Thread.currentThread().setName(StringUtils.format("statement_sql[%s]", sqlQueryId)); + + final DirectStatement.ResultSet plan = stmt.plan(); + // in case the engine is async, the query is not run yet. We just return the taskID in case of non explain queries. + final QueryResponse response = plan.run(); + final Sequence sequence = response.getResults(); + final SqlRowTransformer rowTransformer = plan.createRowTransformer(); + + final boolean isTaskStruct = MSQTaskSqlEngine.TASK_STRUCT_FIELD_NAMES.equals(rowTransformer.getFieldList()); + + if (isTaskStruct) { + return buildTaskResponse(sequence, stmt.query().authResult().getIdentity()); + } else { + // Used for EXPLAIN + return buildStandardResponse(sequence, sqlQuery, sqlQueryId, rowTransformer); + } + } + catch (DruidException e) { + stmt.reporter().failed(e); + return buildNonOkResponse(e); + } + catch (QueryException queryException) { + stmt.reporter().failed(queryException); + final DruidException underlyingException = DruidException.fromFailure(new QueryExceptionCompat(queryException)); + return buildNonOkResponse(underlyingException); + } + catch (ForbiddenException e) { + log.debug("Got forbidden request for reason [%s]", e.getErrorMessage()); + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE) + ); + } + // Calcite throws java.lang.AssertionError at various points in planning/validation. + catch (AssertionError | Exception e) { + stmt.reporter().failed(e); + log.noStackTrace().warn(e, "Failed to handle query: %s", sqlQueryId); + + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e.getMessage()) + ); + } + finally { + stmt.close(); + Thread.currentThread().setName(currThreadName); + } + } + + + @GET + @Path("/{id}") + @Produces(MediaType.APPLICATION_JSON) + public Response doGetStatus( + @PathParam("id") final String queryId, @Context final HttpServletRequest req + ) + { + try { + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + Collections.emptyList(), + authorizerMapper + ); + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); + + Optional sqlStatementResult = getStatementStatus( + queryId, + authenticationResult.getIdentity(), + true + ); + + if (sqlStatementResult.isPresent()) { + return Response.ok().entity(sqlStatementResult.get()).build(); + } else { + return Response.status(Response.Status.NOT_FOUND).build(); + } + } + catch (DruidException e) { + return buildNonOkResponse(e); + } + catch (ForbiddenException e) { + log.debug("Got forbidden request for reason [%s]", e.getErrorMessage()); + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE) + ); + } + catch (Exception e) { + log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); + return buildNonOkResponse(DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Failed to handle query: [%s]", queryId)); + } + } + + @GET + @Path("/{id}/results") + @Produces(MediaType.APPLICATION_JSON) + public Response doGetResults( + @PathParam("id") final String queryId, + @QueryParam("offset") Long offset, + @QueryParam("numRows") Long numberOfRows, + @Context final HttpServletRequest req + ) + { + try { + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + Collections.emptyList(), + authorizerMapper + ); + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); + + if (offset != null && offset < 0) { + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "offset cannot be negative. Please pass a positive number." + ) + ); + } + if (numberOfRows != null && numberOfRows < 0) { + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "numRows cannot be negative. Please pass a positive number." + ) + ); + } + + final long start = offset == null ? 0 : offset; + final long last = SqlStatementResourceHelper.getLastIndex(numberOfRows, start); + + TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId)); + if (taskResponse == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + TaskStatusPlus statusPlus = taskResponse.getStatus(); + if (statusPlus == null || !MSQControllerTask.TYPE.equals(statusPlus.getType())) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); + SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); + + if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + queryId, + sqlStatementState + ) + ); + } else if (sqlStatementState == SqlStatementState.FAILED) { + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] failed. Hit status api for more details.", + queryId + ) + ); + } else { + Optional> signature = SqlStatementResourceHelper.getSignature(msqControllerTask); + if (!signature.isPresent()) { + return Response.ok().build(); + } + Optional> results = SqlStatementResourceHelper.getResults(SqlStatementResourceHelper.getPayload( + contactOverlord(overlordClient.taskReportAsMap(queryId)))); + + return Response.ok((StreamingOutput) outputStream -> { + CountingOutputStream os = new CountingOutputStream(outputStream); + + try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) { + List rowSignature = signature.get(); + writer.writeResponseStart(); + + for (long k = start; k < Math.min(last, results.get().size()); k++) { + writer.writeRowStart(); + for (int i = 0; i < rowSignature.size(); i++) { + writer.writeRowField( + rowSignature.get(i).getColName(), + ((List) results.get().get(Math.toIntExact(k))).get(i) + ); + } + writer.writeRowEnd(); + } + + writer.writeResponseEnd(); + } + catch (Exception e) { + log.error(e, "Unable to stream results back for query[%s]", queryId); + throw new ISE(e, "Unable to stream results back for query[%s]", queryId); + } + }).build(); + + } + } + catch (DruidException e) { + return buildNonOkResponse(e); + } + catch (ForbiddenException e) { + log.debug("Got forbidden request for reason [%s]", e.getErrorMessage()); + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE) + ); + } + catch (Exception e) { + log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); + return buildNonOkResponse(DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Failed to handle query: [%s]", queryId)); + } + } + + /** + * Queries can be canceled while in any {@link SqlStatementState}. Canceling a query that has already completed will be a no-op. + * + * @param queryId queryId + * @param req httpServletRequest + * @return HTTP 404 if the query ID does not exist,expired or originated by different user. HTTP 202 if the deletion + * request has been accepted. + */ + @DELETE + @Path("/{id}") + @Produces(MediaType.APPLICATION_JSON) + public Response deleteQuery(@PathParam("id") final String queryId, @Context final HttpServletRequest req) + { + + try { + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + Collections.emptyList(), + authorizerMapper + ); + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); + + Optional sqlStatementResult = getStatementStatus( + queryId, + authenticationResult.getIdentity(), + false + ); + if (sqlStatementResult.isPresent()) { + switch (sqlStatementResult.get().getState()) { + case ACCEPTED: + case RUNNING: + overlordClient.cancelTask(queryId); + return Response.status(Response.Status.ACCEPTED).build(); + case SUCCESS: + case FAILED: + // we would also want to clean up the results in the future. + return Response.ok().build(); + default: + throw new ISE("Illegal State[%s] encountered", sqlStatementResult.get().getState()); + } + + } else { + return Response.status(Response.Status.NOT_FOUND).build(); + } + } + catch (DruidException e) { + return buildNonOkResponse(e); + } + catch (ForbiddenException e) { + log.debug("Got forbidden request for reason [%s]", e.getErrorMessage()); + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE) + ); + } + catch (Exception e) { + log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); + return buildNonOkResponse(DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Failed to handle query: [%s]", queryId)); + } + } + + private Response buildStandardResponse( + Sequence sequence, + SqlQuery sqlQuery, + String sqlQueryId, + SqlRowTransformer rowTransformer + ) throws IOException + { + final Yielder yielder0 = Yielders.each(sequence); + + try { + final Response.ResponseBuilder responseBuilder = Response.ok((StreamingOutput) outputStream -> { + CountingOutputStream os = new CountingOutputStream(outputStream); + Yielder yielder = yielder0; + + try (final ResultFormat.Writer writer = sqlQuery.getResultFormat().createFormatter(os, jsonMapper)) { + writer.writeResponseStart(); + + if (sqlQuery.includeHeader()) { + writer.writeHeader( + rowTransformer.getRowType(), + sqlQuery.includeTypesHeader(), + sqlQuery.includeSqlTypesHeader() + ); + } + + while (!yielder.isDone()) { + final Object[] row = yielder.get(); + writer.writeRowStart(); + for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { + final Object value = rowTransformer.transform(row, i); + writer.writeRowField(rowTransformer.getFieldList().get(i), value); + } + writer.writeRowEnd(); + yielder = yielder.next(null); + } + + writer.writeResponseEnd(); + } + catch (Exception e) { + log.error(e, "Unable to send SQL response [%s]", sqlQueryId); + throw new RuntimeException(e); + } + finally { + yielder.close(); + } + }); + + if (sqlQuery.includeHeader()) { + responseBuilder.header(SqlResource.SQL_HEADER_RESPONSE_HEADER, SqlResource.SQL_HEADER_VALUE); + } + + return responseBuilder.build(); + } + catch (Throwable e) { + // make sure to close yielder if anything happened before starting to serialize the response. + yielder0.close(); + throw e; + } + } + + private Response buildTaskResponse(Sequence sequence, String user) + { + List rows = sequence.toList(); + int numRows = rows.size(); + if (numRows != 1) { + throw new RE("Expected a single row but got [%d] rows. Please check broker logs for more information.", numRows); + } + Object[] firstRow = rows.get(0); + if (firstRow == null || firstRow.length != 1) { + throw new RE( + "Expected a single column but got [%s] columns. Please check broker logs for more information.", + firstRow == null ? 0 : firstRow.length + ); + } + String taskId = String.valueOf(firstRow[0]); + + Optional statementResult = getStatementStatus(taskId, user, true); + + if (statementResult.isPresent()) { + return Response.status(Response.Status.OK).entity(statementResult.get()).build(); + } else { + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.DEFENSIVE).build( + "Unable to find associated task for query id [%s]. Contact cluster admin to check overlord logs for [%s]", + taskId, + taskId + ) + ); + } + } + + private Response buildNonOkResponse(DruidException exception) + { + return Response + .status(exception.getStatusCode()) + .entity(new ErrorResponse(exception)) + .build(); + } + + private Optional getSampleResults( + String asyncResultId, + boolean isSelectQuery, + String dataSource, + SqlStatementState sqlStatementState + ) + { + if (sqlStatementState == SqlStatementState.SUCCESS) { + Map payload = SqlStatementResourceHelper.getPayload(contactOverlord(overlordClient.taskReportAsMap( + asyncResultId))); + Optional> rowsAndSize = SqlStatementResourceHelper.getRowsAndSizeFromPayload( + payload, + isSelectQuery + ); + return Optional.of(new ResultSetInformation( + null, + // since the rows can be sampled, get the number of rows from counters + rowsAndSize.orElse(new Pair<>(null, null)).lhs, + rowsAndSize.orElse(new Pair<>(null, null)).rhs, + dataSource, + // only populate sample results in case a select query is successful + isSelectQuery ? SqlStatementResourceHelper.getResults(payload).orElse(null) : null + )); + } else { + return Optional.empty(); + } + } + + + private Optional getStatementStatus(String queryId, String currentUser, boolean withResults) + throws DruidException + { + TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId)); + if (taskResponse == null) { + return Optional.empty(); + } + + TaskStatusPlus statusPlus = taskResponse.getStatus(); + if (statusPlus == null || !MSQControllerTask.TYPE.equals(statusPlus.getType())) { + return Optional.empty(); + } + + // since we need the controller payload for auth checks. + MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, currentUser); + SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); + + if (SqlStatementState.FAILED == sqlStatementState) { + return SqlStatementResourceHelper.getExceptionPayload( + queryId, + taskResponse, + statusPlus, + sqlStatementState, + contactOverlord(overlordClient.taskReportAsMap( + queryId)) + ); + } else { + Optional> signature = SqlStatementResourceHelper.getSignature(msqControllerTask); + return Optional.of(new SqlStatementResult( + queryId, + sqlStatementState, + taskResponse.getStatus().getCreatedTime(), + signature.orElse(null), + taskResponse.getStatus().getDuration(), + withResults ? getSampleResults( + queryId, + signature.isPresent(), + msqControllerTask.getDataSource(), + sqlStatementState + ).orElse(null) : null, + null + )); + } + } + + + private MSQControllerTask getMSQControllerTaskOrThrow(String queryId, String currentUser) + { + TaskPayloadResponse taskPayloadResponse = contactOverlord(overlordClient.taskPayload(queryId)); + SqlStatementResourceHelper.isMSQPayload(taskPayloadResponse, queryId); + + MSQControllerTask msqControllerTask = (MSQControllerTask) taskPayloadResponse.getPayload(); + String queryUser = String.valueOf(msqControllerTask.getQuerySpec() + .getQuery() + .getContext() + .get(MSQTaskQueryMaker.USER_KEY)); + if (currentUser == null || !currentUser.equals(queryUser)) { + throw new ForbiddenException(StringUtils.format( + "The current user[%s] cannot view query id[%s] since the query is owned by user[%s]", + currentUser, + queryId, + queryUser + )); + } + return msqControllerTask; + } + + private T contactOverlord(final ListenableFuture future) + { + try { + return FutureUtils.getUnchecked(future, true); + } + catch (RuntimeException e) { + throw DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build("Unable to contact overlord " + e.getMessage()); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java similarity index 98% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java index dd4ec4d2b3c..ecabbd0fa5c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.sql; +package org.apache.druid.msq.sql.resources; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; @@ -32,6 +32,8 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.sql.MSQTaskSqlEngine; +import org.apache.druid.msq.sql.SqlTaskStatus; import org.apache.druid.query.QueryException; import org.apache.druid.server.QueryResponse; import org.apache.druid.server.initialization.ServerConfig; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java new file mode 100644 index 00000000000..4c0474e1f93 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java @@ -0,0 +1,285 @@ +/* + * 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.druid.msq.util; + +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.error.DruidException; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.TaskReportMSQDestination; +import org.apache.druid.msq.sql.SqlStatementState; +import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; +import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.sql.calcite.planner.ColumnMappings; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class SqlStatementResourceHelper +{ + public static Optional> getSignature( + MSQControllerTask msqControllerTask + ) + { + // only populate signature for select q's + if (msqControllerTask.getQuerySpec().getDestination().getClass() == TaskReportMSQDestination.class) { + ColumnMappings columnMappings = msqControllerTask.getQuerySpec().getColumnMappings(); + List sqlTypeNames = msqControllerTask.getSqlTypeNames(); + if (sqlTypeNames == null || sqlTypeNames.size() != columnMappings.size()) { + return Optional.empty(); + } + List nativeTypeNames = msqControllerTask.getNativeTypeNames(); + if (nativeTypeNames == null || nativeTypeNames.size() != columnMappings.size()) { + return Optional.empty(); + } + List signature = new ArrayList<>(columnMappings.size()); + int index = 0; + for (String colName : columnMappings.getOutputColumnNames()) { + signature.add(new ColumnNameAndTypes( + colName, + sqlTypeNames.get(index).getName(), + nativeTypeNames.get(index).asTypeString() + )); + index++; + } + return Optional.of(signature); + } + return Optional.empty(); + } + + + public static void isMSQPayload(TaskPayloadResponse taskPayloadResponse, String queryId) throws DruidException + { + if (taskPayloadResponse == null || taskPayloadResponse.getPayload() == null) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] not found", queryId); + } + + if (MSQControllerTask.class != taskPayloadResponse.getPayload().getClass()) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] not found", queryId); + } + } + + public static SqlStatementState getSqlStatementState(TaskStatusPlus taskStatusPlus) + { + TaskState state = taskStatusPlus.getStatusCode(); + if (state == null) { + return SqlStatementState.ACCEPTED; + } + + switch (state) { + case FAILED: + return SqlStatementState.FAILED; + case RUNNING: + if (TaskLocation.unknown().equals(taskStatusPlus.getLocation())) { + return SqlStatementState.ACCEPTED; + } else { + return SqlStatementState.RUNNING; + } + case SUCCESS: + return SqlStatementState.SUCCESS; + default: + throw new ISE("Unrecognized state[%s] found.", state); + } + } + + @SuppressWarnings("unchecked") + + + public static long getLastIndex(Long numberOfRows, long start) + { + final long last; + if (numberOfRows == null) { + last = Long.MAX_VALUE; + } else { + long finalIndex; + try { + finalIndex = Math.addExact(start, numberOfRows); + } + catch (ArithmeticException e) { + finalIndex = Long.MAX_VALUE; + } + last = finalIndex; + } + return last; + } + + public static Optional> getRowsAndSizeFromPayload(Map payload, boolean isSelectQuery) + { + List stages = getList(payload, "stages"); + if (stages == null || stages.isEmpty()) { + return Optional.empty(); + } else { + int maxStage = stages.size() - 1; // Last stage output is the total number of rows returned to the end user. + Map counterMap = getMap(getMap(payload, "counters"), String.valueOf(maxStage)); + long rows = -1L; + long sizeInBytes = -1L; + if (counterMap == null) { + return Optional.empty(); + } + for (Map.Entry worker : counterMap.entrySet()) { + Object workerChannels = worker.getValue(); + if (workerChannels == null || !(workerChannels instanceof Map)) { + return Optional.empty(); + } + if (isSelectQuery) { + Object output = ((Map) workerChannels).get("output"); + if (output != null && output instanceof Map) { + List rowsPerChannel = (List) ((Map) output).get("rows"); + List bytesPerChannel = (List) ((Map) output).get("bytes"); + for (Integer row : rowsPerChannel) { + rows = rows + row; + } + for (Integer bytes : bytesPerChannel) { + sizeInBytes = sizeInBytes + bytes; + } + } + } else { + Object output = ((Map) workerChannels).get("segmentGenerationProgress"); + if (output != null && output instanceof Map) { + rows += (Integer) ((Map) output).get("rowsPushed"); + } + } + } + + return Optional.of(new Pair<>(rows == -1L ? null : rows + 1, sizeInBytes == -1L ? null : sizeInBytes + 1)); + } + } + + + public static Optional getExceptionPayload( + String queryId, + TaskStatusResponse taskResponse, + TaskStatusPlus statusPlus, + SqlStatementState sqlStatementState, + Map msqPayload + ) + { + Map exceptionDetails = getQueryExceptionDetails(getPayload(msqPayload)); + Map exception = getMap(exceptionDetails, "error"); + if (exceptionDetails == null || exception == null) { + return Optional.of(new SqlStatementResult( + queryId, + sqlStatementState, + taskResponse.getStatus().getCreatedTime(), + null, + taskResponse.getStatus().getDuration(), + null, + DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(taskResponse.getStatus().getErrorMsg()).toErrorResponse() + )); + } + + final String errorMessage = String.valueOf(exception.getOrDefault("errorMessage", statusPlus.getErrorMsg())); + exception.remove("errorMessage"); + String errorCode = String.valueOf(exception.getOrDefault("errorCode", "unknown")); + exception.remove("errorCode"); + Map stringException = new HashMap<>(); + for (Map.Entry exceptionKeys : exception.entrySet()) { + stringException.put(exceptionKeys.getKey(), String.valueOf(exceptionKeys.getValue())); + } + return Optional.of(new SqlStatementResult( + queryId, + sqlStatementState, + taskResponse.getStatus().getCreatedTime(), + null, + taskResponse.getStatus().getDuration(), + null, + DruidException.fromFailure(new DruidException.Failure(errorCode) + { + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + DruidException ex = bob.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(errorMessage); + ex.withContext(stringException); + return ex; + } + }).toErrorResponse() + )); + } + + public static Map getQueryExceptionDetails(Map payload) + { + return getMap(getMap(payload, "status"), "errorReport"); + } + + public static Map getMap(Map map, String key) + { + if (map == null) { + return null; + } + return (Map) map.get(key); + } + + @SuppressWarnings("rawtypes") + public static List getList(Map map, String key) + { + if (map == null) { + return null; + } + return (List) map.get(key); + } + + /** + * Get results from report + */ + @SuppressWarnings("unchecked") + public static Optional> getResults(Map payload) + { + Map resultsHolder = getMap(payload, "results"); + + if (resultsHolder == null) { + return Optional.empty(); + } + + List data = (List) resultsHolder.get("results"); + List rows = new ArrayList<>(); + if (data != null) { + rows.addAll(data); + } + return Optional.of(rows); + } + + public static Map getPayload(Map results) + { + Map msqReport = getMap(results, "multiStageQuery"); + Map payload = getMap(msqReport, "payload"); + return payload; + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java index e1c5c8bf53d..33e0d8c2486 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java @@ -65,6 +65,7 @@ public class MSQControllerTaskTest null, null, null, + null, null); Assert.assertTrue(msqWorkerTask.getInputSourceResources().isEmpty()); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index 6c111c7fd80..b226583922d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -58,7 +58,7 @@ public class MSQTaskReportTest { private static final String TASK_ID = "mytask"; private static final String HOST = "example.com:1234"; - private static final QueryDefinition QUERY_DEFINITION = + public static final QueryDefinition QUERY_DEFINITION = QueryDefinition .builder() .add( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java new file mode 100644 index 00000000000..ceae64dcf70 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -0,0 +1,256 @@ +/* + * 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.druid.msq.sql; + + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; +import org.apache.druid.msq.sql.entity.ResultSetInformation; +import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.msq.sql.resources.SqlStatementResource; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.msq.test.MSQTestOverlordServiceClient; +import org.apache.druid.query.ExecutionMode; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.http.SqlQuery; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.ws.rs.core.Response; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class SqlMsqStatementResourcePostTest extends MSQTestBase +{ + private SqlStatementResource resource; + + @Before + public void init() + { + resource = new SqlStatementResource( + sqlStatementFactory, + CalciteTests.TEST_AUTHORIZER_MAPPER, + objectMapper, + indexingServiceClient + ); + } + + @Test + public void testMSQSelectQueryTest() throws IOException + { + List results = ImmutableList.of( + new Object[]{1L, ""}, + new Object[]{ + 1L, + "10.1" + }, + new Object[]{1L, "2"}, + new Object[]{1L, "1"}, + new Object[]{1L, "def"}, + new Object[]{1L, "abc"} + ); + + Response response = resource.doPost(new SqlQuery( + "select cnt,dim1 from foo", + null, + false, + false, + false, + defaultAsyncContext(), + null + ), SqlStatementResourceTest.makeOkRequest()); + + + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + + String taskId = ((SqlStatementResult) response.getEntity()).getQueryId(); + + SqlStatementResult expected = + new SqlStatementResult(taskId, SqlStatementState.SUCCESS, + MSQTestOverlordServiceClient.CREATED_TIME, + ImmutableList.of( + new ColumnNameAndTypes( + "cnt", + SqlTypeName.BIGINT.getName(), + ValueType.LONG.name() + ), + new ColumnNameAndTypes( + "dim1", + SqlTypeName.VARCHAR.getName(), + ValueType.STRING.name() + ) + ), + MSQTestOverlordServiceClient.DURATION, + new ResultSetInformation( + null, + 6L, + 316L, + MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, + objectMapper.readValue( + objectMapper.writeValueAsString( + results), + new TypeReference>() + { + } + ) + ), + null + ); + + Assert.assertEquals(expected, response.getEntity()); + } + + + @Test + public void nonSupportedModes() + { + for (ImmutableMap context : ImmutableList.of(ImmutableMap.of( + QueryContexts.CTX_EXECUTION_MODE, + ExecutionMode.SYNC.name() + ), ImmutableMap.of())) { + SqlStatementResourceTest.assertExceptionMessage( + resource.doPost(new SqlQuery( + "select * from foo", + null, + false, + false, + false, + (Map) context, + null + ), SqlStatementResourceTest.makeOkRequest()), + "The statement sql api only supports sync mode[ASYNC]. Please set context parameter [executionMode=ASYNC] in the context payload", + Response.Status.BAD_REQUEST + ); + } + } + + + @Test + public void insertCannotBeEmptyFaultTest() + { + Response response = resource.doPost(new SqlQuery( + "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null and __time < TIMESTAMP '1971-01-01 00:00:00' group by 1, 2 PARTITIONED by day clustered by dim1", + null, + false, + false, + false, + defaultAsyncContext(), + null + ), SqlStatementResourceTest.makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + + SqlStatementResult actual = (SqlStatementResult) response.getEntity(); + + InsertCannotBeEmptyFault insertCannotBeEmptyFault = new InsertCannotBeEmptyFault("foo1"); + + MSQException insertCannotBeEmpty = new MSQException(insertCannotBeEmptyFault); + + SqlStatementResult expected = new SqlStatementResult( + actual.getQueryId(), + SqlStatementState.FAILED, + MSQTestOverlordServiceClient.CREATED_TIME, + null, + MSQTestOverlordServiceClient.DURATION, + null, + DruidException.fromFailure(new DruidException.Failure(InsertCannotBeEmptyFault.CODE) + { + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + DruidException e = bob.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(insertCannotBeEmpty.getFault().getErrorMessage()); + e.withContext("dataSource", insertCannotBeEmptyFault.getDataSource()); + return e; + } + }).toErrorResponse() + ); + Assert.assertEquals(expected, actual); + } + + @Test + public void testExplain() throws IOException + { + Map context = defaultAsyncContext(); + context.put("sqlQueryId", "queryId"); + Response response = resource.doPost(new SqlQuery( + "explain plan for select * from foo", + null, + false, + false, + false, + context, + null + ), SqlStatementResourceTest.makeOkRequest()); + + Assert.assertEquals( + "{PLAN=[{\"query\":" + + "{\"queryType\":\"scan\"," + + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," + + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + + "\"resultFormat\":\"compactedList\"," + + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," + + "\"legacy\":false," + + "\"context\":{\"executionMode\":\"ASYNC\",\"scanSignature\":\"[{\\\"name\\\":\\\"__time\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"cnt\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"dim1\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim2\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim3\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"m1\\\",\\\"type\\\":\\\"FLOAT\\\"},{\\\"name\\\":\\\"m2\\\",\\\"type\\\":\\\"DOUBLE\\\"},{\\\"name\\\":\\\"unique_dim1\\\",\\\"type\\\":\\\"COMPLEX\\\"}]\",\"sqlQueryId\":\"queryId\"},\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]," + + " RESOURCES=[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]," + + " ATTRIBUTES={\"statementType\":\"SELECT\"}}", + String.valueOf(SqlStatementResourceTest.getResultRowsFromResponse(response).get(0)) + ); + } + + @Test + public void forbiddenTest() + { + Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resource.doPost( + new SqlQuery( + StringUtils.format("select * from %s", CalciteTests.FORBIDDEN_DATASOURCE), + null, + false, + false, + false, + defaultAsyncContext(), + null + ), + SqlStatementResourceTest.makeOkRequest() + ).getStatus()); + } + + + private static Map defaultAsyncContext() + { + Map context = new HashMap(); + context.put(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()); + return context; + } + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java new file mode 100644 index 00000000000..3f2e7288b62 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -0,0 +1,947 @@ +/* + * 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.druid.msq.sql; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.Futures; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.error.ErrorResponse; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.indexing.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.TaskReportMSQDestination; +import org.apache.druid.msq.indexing.report.MSQResultsReport; +import org.apache.druid.msq.indexing.report.MSQStagesReport; +import org.apache.druid.msq.indexing.report.MSQStatusReport; +import org.apache.druid.msq.indexing.report.MSQTaskReport; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.indexing.report.MSQTaskReportTest; +import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; +import org.apache.druid.msq.sql.entity.ResultSetInformation; +import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.msq.sql.resources.SqlStatementResource; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.query.Druids; +import org.apache.druid.query.Query; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.server.mocks.MockHttpServletRequest; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.sql.calcite.planner.ColumnMappings; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.http.SqlResourceTest; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mock; +import org.mockito.Mockito; + +import javax.ws.rs.core.Response; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class SqlStatementResourceTest extends MSQTestBase +{ + + public static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); + private static final String ACCEPTED_SELECT_MSQ_QUERY = "QUERY_ID_1"; + private static final String RUNNING_SELECT_MSQ_QUERY = "QUERY_ID_2"; + private static final String FINISHED_SELECT_MSQ_QUERY = "QUERY_ID_3"; + + private static final String ERRORED_SELECT_MSQ_QUERY = "QUERY_ID_4"; + + + private static final String RUNNING_NON_MSQ_TASK = "QUERY_ID_5"; + + private static final String FAILED_NON_MSQ_TASK = "QUERY_ID_6"; + + private static final String FINISHED_NON_MSQ_TASK = "QUERY_ID_7"; + + + private static final String ACCEPTED_INSERT_MSQ_TASK = "QUERY_ID_8"; + + private static final String RUNNING_INSERT_MSQ_QUERY = "QUERY_ID_9"; + private static final String FINISHED_INSERT_MSQ_QUERY = "QUERY_ID_10"; + private static final String ERRORED_INSERT_MSQ_QUERY = "QUERY_ID_11"; + + + private static final Query QUERY = new Druids.ScanQueryBuilder().resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .intervals(new MultipleIntervalSegmentSpec( + Collections.singletonList(Intervals.of( + "2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z")))) + .dataSource("target") + .context(ImmutableMap.of( + MSQTaskQueryMaker.USER_KEY, + AuthConfig.ALLOW_ALL_NAME + )) + .build(); + + + private static final MSQControllerTask MSQ_CONTROLLER_SELECT_PAYLOAD = new MSQControllerTask( + ACCEPTED_SELECT_MSQ_QUERY, + MSQSpec.builder() + .query(QUERY) + .columnMappings( + ColumnMappings.identity( + RowSignature.builder() + .add( + "_time", + ColumnType.LONG + ) + .add( + "alias", + ColumnType.STRING + ) + .add( + "market", + ColumnType.STRING + ) + .build())) + .destination( + TaskReportMSQDestination.INSTANCE) + .tuningConfig( + MSQTuningConfig.defaultConfig()) + .build(), + "select _time,alias,market from test", + new HashMap<>(), + null, + ImmutableList.of( + SqlTypeName.TIMESTAMP, + SqlTypeName.VARCHAR, + SqlTypeName.VARCHAR + ), + ImmutableList.of( + ColumnType.LONG, + ColumnType.STRING, + ColumnType.STRING + ), + null + ); + + private static final MSQControllerTask MSQ_CONTROLLER_INSERT_PAYLOAD = new MSQControllerTask( + ACCEPTED_SELECT_MSQ_QUERY, + MSQSpec.builder() + .query(QUERY) + .columnMappings( + ColumnMappings.identity( + RowSignature.builder() + .add( + "_time", + ColumnType.LONG + ) + .add( + "alias", + ColumnType.STRING + ) + .add( + "market", + ColumnType.STRING + ) + .build())) + .destination(new DataSourceMSQDestination( + "test", + Granularities.DAY, + null, + null + )) + .tuningConfig( + MSQTuningConfig.defaultConfig()) + .build(), + "insert into test select _time,alias,market from test", + new HashMap<>(), + null, + ImmutableList.of( + SqlTypeName.TIMESTAMP, + SqlTypeName.VARCHAR, + SqlTypeName.VARCHAR + ), + ImmutableList.of( + ColumnType.LONG, + ColumnType.STRING, + ColumnType.STRING + ), + null + ); + + private static final List RESULT_ROWS = ImmutableList.of( + new Object[]{123, "foo", "bar"}, + new Object[]{234, "foo1", "bar1"} + ); + + private static final MSQTaskReport MSQ_SELECT_TASK_REPORT = new MSQTaskReport( + FINISHED_SELECT_MSQ_QUERY, + new MSQTaskReportPayload( + new MSQStatusReport( + TaskState.SUCCESS, + null, + new ArrayDeque<>(), + null, + 0, + 1, + 2 + ), + MSQStagesReport.create( + MSQTaskReportTest.QUERY_DEFINITION, + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of() + ), + new CounterSnapshotsTree(), + new MSQResultsReport( + ImmutableList.of( + new MSQResultsReport.ColumnAndType( + "_time", + ColumnType.LONG + ), + new MSQResultsReport.ColumnAndType( + "alias", + ColumnType.STRING + ), + new MSQResultsReport.ColumnAndType( + "market", + ColumnType.STRING + ) + ), + ImmutableList.of( + SqlTypeName.TIMESTAMP, + SqlTypeName.VARCHAR, + SqlTypeName.VARCHAR + ), + Yielders.each( + Sequences.simple( + RESULT_ROWS)), + null + ) + ) + ); + + private static final MSQTaskReport MSQ_INSERT_TASK_REPORT = new MSQTaskReport( + FINISHED_INSERT_MSQ_QUERY, + new MSQTaskReportPayload( + new MSQStatusReport( + TaskState.SUCCESS, + null, + new ArrayDeque<>(), + null, + 0, + 1, + 2 + ), + MSQStagesReport.create( + MSQTaskReportTest.QUERY_DEFINITION, + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of() + ), + new CounterSnapshotsTree(), + null + ) + ); + private static final DateTime QUEUE_INSERTION_TIME = DateTimes.of("2023-05-31T12:01Z"); + private static final Map ROW1 = ImmutableMap.of("_time", 123, "alias", "foo", "market", "bar"); + private static final Map ROW2 = ImmutableMap.of("_time", 234, "alias", "foo1", "market", "bar1"); + public static final ImmutableList COL_NAME_AND_TYPES = ImmutableList.of( + new ColumnNameAndTypes( + "_time", + SqlTypeName.TIMESTAMP.getName(), + ValueType.LONG.name() + ), + new ColumnNameAndTypes( + "alias", + SqlTypeName.VARCHAR.getName(), + ValueType.STRING.name() + ), + new ColumnNameAndTypes( + "market", + SqlTypeName.VARCHAR.getName(), + ValueType.STRING.name() + ) + ); + private static final String FAILURE_MSG = "failure msg"; + private static SqlStatementResource resource; + @Mock + private static OverlordClient overlordClient; + + private static void setupMocks(OverlordClient indexingServiceClient) throws JsonProcessingException + { + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(ACCEPTED_SELECT_MSQ_QUERY, new TaskStatusPlus( + ACCEPTED_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + null, + null, + null, + TaskLocation.unknown(), + null, + null + )))); + + Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + ACCEPTED_SELECT_MSQ_QUERY, + MSQ_CONTROLLER_SELECT_PAYLOAD + ))); + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(RUNNING_SELECT_MSQ_QUERY, new TaskStatusPlus( + RUNNING_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.RUNNING, + null, + null, + new TaskLocation("test", 0, 0), + null, + null + )))); + + Mockito.when(indexingServiceClient.taskPayload(RUNNING_SELECT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + RUNNING_SELECT_MSQ_QUERY, + MSQ_CONTROLLER_SELECT_PAYLOAD + ))); + + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FINISHED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(FINISHED_SELECT_MSQ_QUERY, new TaskStatusPlus( + FINISHED_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + 100L, + new TaskLocation("test", 0, 0), + null, + null + )))); + + Mockito.when(indexingServiceClient.taskPayload(FINISHED_SELECT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + FINISHED_SELECT_MSQ_QUERY, + MSQ_CONTROLLER_SELECT_PAYLOAD + ))); + + + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + + Mockito.when(indexingServiceClient.taskReportAsMap(FINISHED_SELECT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(mapper.readValue( + mapper.writeValueAsString(TaskReport.buildTaskReports( + MSQ_SELECT_TASK_REPORT)), + new TypeReference>() + { + } + ))); + + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(ERRORED_SELECT_MSQ_QUERY, new TaskStatusPlus( + ERRORED_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.FAILED, + null, + -1L, + TaskLocation.unknown(), + null, + FAILURE_MSG + )))); + + Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + FINISHED_INSERT_MSQ_QUERY, + MSQ_CONTROLLER_SELECT_PAYLOAD + ))); + + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(null)); + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_NON_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(RUNNING_NON_MSQ_TASK, new TaskStatusPlus( + RUNNING_NON_MSQ_TASK, + null, + null, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.RUNNING, + null, + -1L, + TaskLocation.unknown(), + null, + null + )))); + + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FAILED_NON_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(FAILED_NON_MSQ_TASK, new TaskStatusPlus( + FAILED_NON_MSQ_TASK, + null, + null, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.FAILED, + null, + -1L, + TaskLocation.unknown(), + null, + FAILURE_MSG + )))); + + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FINISHED_NON_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(FINISHED_NON_MSQ_TASK, new TaskStatusPlus( + FINISHED_NON_MSQ_TASK, + null, + IndexTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + -1L, + TaskLocation.unknown(), + null, + null + )))); + + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(ACCEPTED_SELECT_MSQ_QUERY, new TaskStatusPlus( + ACCEPTED_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + null, + null, + null, + TaskLocation.unknown(), + null, + null + )))); + + Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + ACCEPTED_INSERT_MSQ_TASK, + MSQ_CONTROLLER_INSERT_PAYLOAD + ))); + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(RUNNING_INSERT_MSQ_QUERY, new TaskStatusPlus( + RUNNING_INSERT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.RUNNING, + null, + null, + new TaskLocation("test", 0, 0), + null, + null + )))); + + Mockito.when(indexingServiceClient.taskPayload(RUNNING_INSERT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + RUNNING_INSERT_MSQ_QUERY, + MSQ_CONTROLLER_INSERT_PAYLOAD + ))); + + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(FINISHED_INSERT_MSQ_QUERY, new TaskStatusPlus( + FINISHED_INSERT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + 100L, + new TaskLocation("test", 0, 0), + null, + null + )))); + + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(mapper.readValue( + mapper.writeValueAsString(TaskReport.buildTaskReports( + MSQ_INSERT_TASK_REPORT)), + new TypeReference>() + { + } + ))); + + Mockito.when(indexingServiceClient.taskPayload(FINISHED_INSERT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + FINISHED_INSERT_MSQ_QUERY, + MSQ_CONTROLLER_INSERT_PAYLOAD + ))); + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(ERRORED_INSERT_MSQ_QUERY, new TaskStatusPlus( + ERRORED_INSERT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.FAILED, + null, + -1L, + TaskLocation.unknown(), + null, + FAILURE_MSG + )))); + + Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + ERRORED_INSERT_MSQ_QUERY, + MSQ_CONTROLLER_INSERT_PAYLOAD + ))); + + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(null)); + + } + + public static void assertNullResponse(Response response, Response.Status expectectedStatus) + { + Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); + Assert.assertNull(response.getEntity()); + } + + public static void assertExceptionMessage( + Response response, + String exceptionMessage, + Response.Status expectectedStatus + ) + { + Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); + Assert.assertEquals(exceptionMessage, getQueryExceptionFromResponse(response)); + } + + public static List getResultRowsFromResponse(Response resultsResponse) throws IOException + { + byte[] bytes = SqlResourceTest.responseToByteArray(resultsResponse); + if (bytes == null) { + return null; + } + return JSON_MAPPER.readValue(bytes, List.class); + } + + private static String getQueryExceptionFromResponse(Response response) + { + if (response.getEntity() instanceof SqlStatementResult) { + return ((SqlStatementResult) response.getEntity()).getErrorResponse().getUnderlyingException().getMessage(); + } else { + return ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage(); + } + } + + public static MockHttpServletRequest makeOkRequest() + { + return makeExpectedReq(CalciteTests.REGULAR_USER_AUTH_RESULT); + } + + public static MockHttpServletRequest makeExpectedReq(AuthenticationResult authenticationResult) + { + MockHttpServletRequest req = new MockHttpServletRequest(); + req.attributes.put(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult); + req.remoteAddr = "1.2.3.4"; + return req; + } + + @Before + public void init() throws Exception + { + overlordClient = Mockito.mock(OverlordClient.class); + setupMocks(overlordClient); + resource = new SqlStatementResource( + sqlStatementFactory, + CalciteTests.TEST_AUTHORIZER_MAPPER, + JSON_MAPPER, + overlordClient + ); + } + + @Test + public void testMSQSelectAcceptedQuery() + { + Response response = resource.doGetStatus(ACCEPTED_SELECT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals( + new SqlStatementResult( + ACCEPTED_SELECT_MSQ_QUERY, + SqlStatementState.ACCEPTED, + CREATED_TIME, + COL_NAME_AND_TYPES, + null, + null, + null + ), + response.getEntity() + ); + + assertExceptionMessage( + resource.doGetResults(ACCEPTED_SELECT_MSQ_QUERY, null, null, makeOkRequest()), + StringUtils.format( + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + ACCEPTED_SELECT_MSQ_QUERY, + SqlStatementState.ACCEPTED + ), + Response.Status.BAD_REQUEST + ); + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(ACCEPTED_SELECT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + } + + @Test + + public void testMSQSelectRunningQuery() + { + + Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals( + new SqlStatementResult( + RUNNING_SELECT_MSQ_QUERY, + SqlStatementState.RUNNING, + CREATED_TIME, + COL_NAME_AND_TYPES, + null, + null, + null + ), + response.getEntity() + ); + + assertExceptionMessage( + resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, null, null, makeOkRequest()), + StringUtils.format( + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + RUNNING_SELECT_MSQ_QUERY, + SqlStatementState.RUNNING + ), + Response.Status.BAD_REQUEST + ); + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + } + + @Test + public void testFinishedSelectMSQQuery() throws Exception + { + Response response = resource.doGetStatus(FINISHED_SELECT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals(new SqlStatementResult( + FINISHED_SELECT_MSQ_QUERY, + SqlStatementState.SUCCESS, + CREATED_TIME, + COL_NAME_AND_TYPES, + 100L, + new ResultSetInformation( + null, + null, + null, + MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, + RESULT_ROWS.stream() + .map(Arrays::asList) + .collect(Collectors.toList()) + ), + null + ), response.getEntity()); + + Response resultsResponse = resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, null, null, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); + + List> rows = new ArrayList<>(); + rows.add(ROW1); + rows.add(ROW2); + + Assert.assertEquals(rows, getResultRowsFromResponse(resultsResponse)); + + Assert.assertEquals( + Response.Status.OK.getStatusCode(), + resource.deleteQuery(FINISHED_SELECT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + + Assert.assertEquals( + rows.subList(1, 2), + getResultRowsFromResponse(resource.doGetResults( + FINISHED_SELECT_MSQ_QUERY, + 1L, + null, + makeOkRequest() + )) + ); + Assert.assertEquals( + rows.subList(0, 1), + getResultRowsFromResponse(resource.doGetResults( + FINISHED_SELECT_MSQ_QUERY, + 0L, + 1L, + makeOkRequest() + )) + ); + Assert.assertEquals( + rows, + getResultRowsFromResponse(resource.doGetResults( + FINISHED_SELECT_MSQ_QUERY, + 0L, + 3L, + makeOkRequest() + )) + ); + + Assert.assertEquals( + Response.Status.BAD_REQUEST.getStatusCode(), + resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, -1L, 3L, makeOkRequest()).getStatus() + ); + Assert.assertEquals( + Response.Status.BAD_REQUEST.getStatusCode(), + resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, null, -1L, makeOkRequest()).getStatus() + ); + + } + + @Test + public void testFailedMSQQuery() + { + for (String queryID : ImmutableList.of(ERRORED_SELECT_MSQ_QUERY, ERRORED_INSERT_MSQ_QUERY)) { + assertExceptionMessage(resource.doGetStatus(queryID, makeOkRequest()), FAILURE_MSG, Response.Status.OK); + assertExceptionMessage( + resource.doGetResults(queryID, null, null, makeOkRequest()), + StringUtils.format( + "Query[%s] failed. Hit status api for more details.", + queryID + ), + Response.Status.BAD_REQUEST + ); + + Assert.assertEquals( + Response.Status.OK.getStatusCode(), + resource.deleteQuery(queryID, makeOkRequest()).getStatus() + ); + } + } + + @Test + public void testFinishedInsertMSQQuery() throws Exception + { + Response response = resource.doGetStatus(FINISHED_INSERT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals(new SqlStatementResult( + FINISHED_INSERT_MSQ_QUERY, + SqlStatementState.SUCCESS, + CREATED_TIME, + null, + 100L, + new ResultSetInformation(null, null, null, "test", null), + null + ), response.getEntity()); + + Response resultsResponse = resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, null, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); + + + Assert.assertNull(getResultRowsFromResponse(resource.doGetResults( + FINISHED_INSERT_MSQ_QUERY, + 1L, + null, + makeOkRequest() + ))); + Assert.assertNull(getResultRowsFromResponse(resource.doGetResults( + FINISHED_INSERT_MSQ_QUERY, + 0L, + 1L, + makeOkRequest() + ))); + Assert.assertNull(getResultRowsFromResponse(resource.doGetResults( + FINISHED_INSERT_MSQ_QUERY, + 0L, + 3L, + makeOkRequest() + ))); + + Assert.assertEquals( + Response.Status.BAD_REQUEST.getStatusCode(), + resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, -1L, 3L, makeOkRequest()).getStatus() + ); + Assert.assertEquals( + Response.Status.BAD_REQUEST.getStatusCode(), + resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, -1L, makeOkRequest()).getStatus() + ); + + } + + @Test + public void testNonMSQTasks() + { + for (String queryID : ImmutableList.of(RUNNING_NON_MSQ_TASK, FAILED_NON_MSQ_TASK, FINISHED_NON_MSQ_TASK)) { + assertNullResponse(resource.doGetStatus(queryID, makeOkRequest()), Response.Status.NOT_FOUND); + assertNullResponse(resource.doGetResults(queryID, null, null, makeOkRequest()), Response.Status.NOT_FOUND); + assertNullResponse(resource.deleteQuery(queryID, makeOkRequest()), Response.Status.NOT_FOUND); + } + } + + @Test + public void testMSQInsertAcceptedQuery() + { + Response response = resource.doGetStatus(ACCEPTED_INSERT_MSQ_TASK, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals( + new SqlStatementResult( + ACCEPTED_INSERT_MSQ_TASK, + SqlStatementState.ACCEPTED, + CREATED_TIME, + null, + null, + null, + null + ), + response.getEntity() + ); + + assertExceptionMessage( + resource.doGetResults(ACCEPTED_INSERT_MSQ_TASK, null, null, makeOkRequest()), + StringUtils.format( + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + ACCEPTED_INSERT_MSQ_TASK, + SqlStatementState.ACCEPTED + ), + Response.Status.BAD_REQUEST + ); + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(ACCEPTED_INSERT_MSQ_TASK, makeOkRequest()).getStatus() + ); + } + + @Test + public void testMSQInsertRunningQuery() + { + Response response = resource.doGetStatus(RUNNING_INSERT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals( + new SqlStatementResult( + RUNNING_INSERT_MSQ_QUERY, + SqlStatementState.RUNNING, + CREATED_TIME, + null, + null, + null, + null + ), + response.getEntity() + ); + + assertExceptionMessage( + resource.doGetResults(RUNNING_INSERT_MSQ_QUERY, null, null, makeOkRequest()), + StringUtils.format( + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + RUNNING_INSERT_MSQ_QUERY, + SqlStatementState.RUNNING + ), + Response.Status.BAD_REQUEST + ); + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(RUNNING_INSERT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + } + + @Test + public void forbiddenTests() + { + + Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), + resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, + makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); + + Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), + resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, + null, + null, + makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); + Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), + resource.deleteQuery(RUNNING_SELECT_MSQ_QUERY, + makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); + } + + @Test + public void testIsEnabled() + { + Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.isEnabled(makeOkRequest()).getStatus()); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypesTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypesTest.java new file mode 100644 index 00000000000..f0524849995 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypesTest.java @@ -0,0 +1,53 @@ +/* + * 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.druid.msq.sql.entity; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +public class ColumnNameAndTypesTest +{ + public static final ObjectMapper MAPPER = new ObjectMapper(); + + public static final ColumnNameAndTypes COLUMN_NAME_AND_TYPES = new ColumnNameAndTypes("test", "test1", "test2"); + public static final String JSON_STRING = "{\"name\":\"test\",\"type\":\"test1\",\"nativeType\":\"test2\"}"; + + @Test + public void sanityTest() throws JsonProcessingException + { + Assert.assertEquals(JSON_STRING, MAPPER.writeValueAsString(COLUMN_NAME_AND_TYPES)); + Assert.assertEquals( + COLUMN_NAME_AND_TYPES, + MAPPER.readValue(MAPPER.writeValueAsString(COLUMN_NAME_AND_TYPES), ColumnNameAndTypes.class) + ); + + Assert.assertEquals( + COLUMN_NAME_AND_TYPES.hashCode(), + MAPPER.readValue(MAPPER.writeValueAsString(COLUMN_NAME_AND_TYPES), ColumnNameAndTypes.class) + .hashCode() + ); + Assert.assertEquals("ColumnNameAndTypes{colName='test', sqlTypeName='test1', nativeTypeName='test2'}", + COLUMN_NAME_AND_TYPES.toString()); + + } + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java new file mode 100644 index 00000000000..14d04b1b76d --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java @@ -0,0 +1,58 @@ +/* + * 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.druid.msq.sql.entity; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.sql.http.ResultFormat; +import org.junit.Assert; +import org.junit.Test; + +public class ResultSetInformationTest +{ + public static final ObjectMapper MAPPER = new ObjectMapper(); + + public static final ResultSetInformation RESULTS = new ResultSetInformation(ResultFormat.OBJECT, 1L, 1L, "ds", + ImmutableList.of( + ImmutableList.of("1"), + ImmutableList.of("2"), + ImmutableList.of("3") + ) + ); + public static final String JSON_STRING = "{\"resultFormat\":\"object\",\"numRows\":1,\"sizeInBytes\":1,\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]]}"; + + + @Test + public void sanityTest() throws JsonProcessingException + { + Assert.assertEquals(JSON_STRING, MAPPER.writeValueAsString(RESULTS)); + Assert.assertEquals(RESULTS, MAPPER.readValue(MAPPER.writeValueAsString(RESULTS), ResultSetInformation.class)); + Assert.assertEquals( + RESULTS.hashCode(), + MAPPER.readValue(MAPPER.writeValueAsString(RESULTS), ResultSetInformation.class).hashCode() + ); + Assert.assertEquals( + "ResultSetInformation{totalRows=1, totalSize=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds'}", + RESULTS.toString() + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java new file mode 100644 index 00000000000..a0be3afcf74 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java @@ -0,0 +1,95 @@ +/* + * 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.druid.msq.sql.entity; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.error.DruidException; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.QueryNotSupportedFault; +import org.apache.druid.msq.sql.SqlStatementResourceTest; +import org.apache.druid.msq.sql.SqlStatementState; +import org.junit.Assert; +import org.junit.Test; + +public class SqlStatementResultTest +{ + public static final MSQException MSQ_EXCEPTION = new MSQException( + QueryNotSupportedFault.instance()); + + public static final ObjectMapper MAPPER = DefaultObjectMapper.INSTANCE; + + public static final String JSON_STRING = "{\"queryId\":\"q1\"," + + "\"state\":\"RUNNING\"," + + "\"createdAt\":\"2023-05-31T12:00:00.000Z\"," + + "\"schema\":[{\"name\":\"_time\",\"type\":\"TIMESTAMP\",\"nativeType\":\"LONG\"},{\"name\":\"alias\",\"type\":\"VARCHAR\",\"nativeType\":\"STRING\"},{\"name\":\"market\",\"type\":\"VARCHAR\",\"nativeType\":\"STRING\"}]," + + "\"durationMs\":100," + + "\"result\":{\"resultFormat\":\"object\",\"numRows\":1,\"sizeInBytes\":1,\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]]}," + + "\"errorDetails\":{\"error\":\"druidException\",\"errorCode\":\"QueryNotSupported\",\"persona\":\"USER\",\"category\":\"UNCATEGORIZED\",\"errorMessage\":\"QueryNotSupported\",\"context\":{}}}"; + + public static final SqlStatementResult SQL_STATEMENT_RESULT = new SqlStatementResult( + "q1", + SqlStatementState.RUNNING, + SqlStatementResourceTest.CREATED_TIME, + SqlStatementResourceTest.COL_NAME_AND_TYPES, + 100L, + ResultSetInformationTest.RESULTS, + DruidException.fromFailure(new DruidException.Failure(MSQ_EXCEPTION.getFault().getErrorCode()) + { + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + DruidException ex = bob.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(MSQ_EXCEPTION.getMessage()); + return ex; + } + }).toErrorResponse() + ); + + + @Test + public void sanityTest() throws JsonProcessingException + { + + Assert.assertEquals(JSON_STRING, MAPPER.writeValueAsString(SQL_STATEMENT_RESULT)); + Assert.assertEquals( + SQL_STATEMENT_RESULT, + MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class) + ); + Assert.assertEquals( + SQL_STATEMENT_RESULT.hashCode(), + MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class).hashCode() + ); + Assert.assertEquals( + "SqlStatementResult{" + + "queryId='q1'," + + " state=RUNNING," + + " createdAt=2023-05-31T12:00:00.000Z," + + " sqlRowSignature=[ColumnNameAndTypes{colName='_time', sqlTypeName='TIMESTAMP', nativeTypeName='LONG'}, ColumnNameAndTypes{colName='alias', sqlTypeName='VARCHAR', nativeTypeName='STRING'}, ColumnNameAndTypes{colName='market', sqlTypeName='VARCHAR', nativeTypeName='STRING'}]," + + " durationInMs=100," + + " resultSetInformation=ResultSetInformation{totalRows=1, totalSize=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds'}," + + " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}}}", + SQL_STATEMENT_RESULT.toString() + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 4301a68b4f3..0138b957b32 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -134,7 +134,6 @@ import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.server.security.AuthConfig; -import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.sql.DirectStatement; import org.apache.druid.sql.SqlQueryPlus; import org.apache.druid.sql.SqlStatementFactory; @@ -226,6 +225,7 @@ public class MSQTestBase extends BaseCalciteQueryTest .put(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false) .put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 2) .put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 0) + .put(MSQTaskQueryMaker.USER_KEY, "allowAll") .build(); public static final Map DURABLE_STORAGE_MSQ_CONTEXT = @@ -266,10 +266,10 @@ public class MSQTestBase extends BaseCalciteQueryTest protected File localFileStorageDir; protected LocalFileStorageConnector localFileStorageConnector; private static final Logger log = new Logger(MSQTestBase.class); - private ObjectMapper objectMapper; - private MSQTestOverlordServiceClient indexingServiceClient; + protected ObjectMapper objectMapper; + protected MSQTestOverlordServiceClient indexingServiceClient; protected MSQTestTaskActionClient testTaskActionClient; - private SqlStatementFactory sqlStatementFactory; + protected SqlStatementFactory sqlStatementFactory; private IndexIO indexIO; private MSQTestSegmentManager segmentManager; @@ -493,7 +493,7 @@ public class MSQTestBase extends BaseCalciteQueryTest qf.operatorTable(), qf.macroTable(), PLANNER_CONFIG_DEFAULT, - AuthTestUtils.TEST_AUTHORIZER_MAPPER, + CalciteTests.TEST_EXTERNAL_AUTHORIZER_MAPPER, objectMapper, CalciteTests.DRUID_SCHEMA_NAME, new CalciteRulesManager(ImmutableSet.of()), @@ -952,10 +952,12 @@ public class MSQTestBase extends BaseCalciteQueryTest worker, channel ); - Assert.assertTrue(StringUtils.format("Counters not found for stage [%d], worker [%d], channel [%s]", - stage, - worker, - channel), channelToCounters.containsKey(channel)); + Assert.assertTrue(StringUtils.format( + "Counters not found for stage [%d], worker [%d], channel [%s]", + stage, + worker, + channel + ), channelToCounters.containsKey(channel)); counter.matchQuerySnapshot(errorMessageFormat, channelToCounters.get(channel)); } ); @@ -1066,7 +1068,7 @@ public class MSQTestBase extends BaseCalciteQueryTest verifyWorkerCount(reportPayload.getCounters()); verifyCounters(reportPayload.getCounters()); - MSQSpec foundSpec = indexingServiceClient.getQuerySpecForTask(controllerId); + MSQSpec foundSpec = indexingServiceClient.getMSQControllerTask(controllerId).getQuerySpec(); log.info( "found generated segments: %s", segmentManager.getAllDataSegments().stream().map(s -> s.toString()).collect( @@ -1285,7 +1287,7 @@ public class MSQTestBase extends BaseCalciteQueryTest log.info("found row signature %s", payload.getResults().getSignature()); log.info(rows.stream().map(Arrays::toString).collect(Collectors.joining("\n"))); - final MSQSpec spec = indexingServiceClient.getQuerySpecForTask(controllerId); + final MSQSpec spec = indexingServiceClient.getMSQControllerTask(controllerId).getQuerySpec(); log.info("Found spec: %s", objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(spec)); return new Pair<>(spec, Pair.of(payload.getResults().getSignature(), rows)); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java index a4aff4d8756..1b49982cad4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java @@ -19,24 +19,29 @@ package org.apache.druid.msq.test; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Injector; import org.apache.druid.client.indexing.NoopOverlordClient; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerImpl; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.indexing.MSQControllerTask; -import org.apache.druid.msq.indexing.MSQSpec; -import org.apache.druid.msq.indexing.report.MSQTaskReport; +import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -48,7 +53,13 @@ public class MSQTestOverlordServiceClient extends NoopOverlordClient private final WorkerMemoryParameters workerMemoryParameters; private Map inMemoryControllers = new HashMap<>(); private Map> reports = new HashMap<>(); - private Map msqSpec = new HashMap<>(); + private Map inMemoryControllerTask = new HashMap<>(); + private Map inMemoryTaskStatus = new HashMap<>(); + + public static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); + public static final DateTime QUEUE_INSERTION_TIME = DateTimes.of("2023-05-31T12:01Z"); + + public static final long DURATION = 100L; public MSQTestOverlordServiceClient( ObjectMapper objectMapper, @@ -77,16 +88,13 @@ public class MSQTestOverlordServiceClient extends NoopOverlordClient ); MSQControllerTask cTask = objectMapper.convertValue(taskObject, MSQControllerTask.class); - msqSpec.put(cTask.getId(), cTask.getQuerySpec()); + inMemoryControllerTask.put(cTask.getId(), cTask); - controller = new ControllerImpl( - cTask, - msqTestControllerContext - ); + controller = new ControllerImpl(cTask, msqTestControllerContext); - inMemoryControllers.put(cTask.getId(), controller); + inMemoryControllers.put(controller.id(), controller); - controller.run(); + inMemoryTaskStatus.put(taskId, controller.run()); return Futures.immediateFuture(null); } catch (Exception e) { @@ -110,24 +118,61 @@ public class MSQTestOverlordServiceClient extends NoopOverlordClient public ListenableFuture> taskReportAsMap(String taskId) { SettableFuture> future = SettableFuture.create(); - future.set( - ImmutableMap.of( - MSQTaskReport.REPORT_KEY, - getReportForTask(taskId).get(MSQTaskReport.REPORT_KEY) - )); + try { + future.set( + objectMapper.readValue( + objectMapper.writeValueAsBytes(getReportForTask(taskId)), + new TypeReference>() + { + } + )); + } + catch (IOException e) { + throw new RuntimeException(e); + } + return future; + } + + @Override + public ListenableFuture taskPayload(String taskId) + { + SettableFuture future = SettableFuture.create(); + future.set(new TaskPayloadResponse(taskId, getMSQControllerTask(taskId))); + return future; + } + + @Override + public ListenableFuture taskStatus(String taskId) + { + SettableFuture future = SettableFuture.create(); + TaskStatus taskStatus = inMemoryTaskStatus.get(taskId); + future.set(new TaskStatusResponse(taskId, new TaskStatusPlus( + taskId, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + taskStatus.getStatusCode(), + null, + DURATION, + taskStatus.getLocation(), + null, + taskStatus.getErrorMsg() + ))); + return future; } // hooks to pull stuff out for testing @Nullable - Map getReportForTask(String id) + public Map getReportForTask(String id) { return reports.get(id); } @Nullable - MSQSpec getQuerySpecForTask(String id) + MSQControllerTask getMSQControllerTask(String id) { - return msqSpec.get(id); + return inMemoryControllerTask.get(id); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 5e3d7b65fbf..9fbf8a6b50b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -144,8 +144,8 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler public static final HashFunction HASH_FUNCTION = Hashing.murmur3_128(); + public static final String TYPE = "index"; private static final Logger log = new Logger(IndexTask.class); - private static final String TYPE = "index"; private static String makeGroupId(IndexIngestionSpec ingestionSchema, IngestionMode ingestionMode) { diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index 81a7939ec4a..85aa73ad1d5 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -47,34 +47,33 @@ import java.util.Map; * no change should occur. * * Notes about exception messages - * + *

* Firstly, exception messages should always be written with the notions from the style conventions covered in * {@code dev/style-conventions.md}. Whenever possible, we should also try to provide an action to take to resolve * the issue. - * + *

* Secondly, given that the DruidException requires defining a target persona, exception messages should always be * written with that target persona in mind. Reviewers should use the targetPersona as added input to help validate * that an exception message in meaningful. - * + *

* For example, at the time that this exception was introduced, there is an exception that the router throws which is * an {@link org.apache.druid.java.util.common.ISE} with the message {@code "No default server found!"}. This * exception is thrown when the router is unable to find a broker to forward a request to. It is completely * meaningless to an end-user trying to run a query (what's a default server? why does it need to be found?). If we - * were to convert the exception to a DruidException and keep the same message, we should mark it as targetting the + * were to convert the exception to a DruidException and keep the same message, we should mark it as targeting the * DEVELOPER persona as that is the only persona who should actually be able to figure out what a default server is - * and why it is important. That said, does it makes sense for an exception that means "router cannot find a broker - * to forward the query to" to only be targetting the DEVELOPER? The answer to that is no, it's something that should - * really be made meaningful to a wider group. Some options could be - * - * USER persona: Cannot find a queryable server, contact your cluster administrator to validate that all services are - * operational - * - * OPERATOR persona: Router unable to find a broker, check that brokers are up and active - * + * and why it is important. That said, does it make sense for an exception that means "router cannot find a broker + * to forward the query to" to only be targeting the DEVELOPER? The answer to that is no, it's something that should + * really be made meaningful to a wider group. Some options could be + *

    + *
  • USER persona: Cannot find a queryable server, contact your cluster administrator to validate that all services are + * operational
  • + *
  • OPERATOR persona: Router unable to find a broker, check that brokers are up and active
  • + *
* The user-facing message doesn't talk about any Druid-specific concepts and just tries to relay a high-level * understanding of what happened. The admin-facing message includes Druid notions in it as it expects that an Admin * will understand the various node types of Druid. - * + *

* If we think about this error more, we will realize that it's fundamentally something wrong with the cluster setup, * which is something that we would expect an operator to be in charge of. So, we would pick the OPERATOR persona * message, which also allows us to include more specific information about what server was not found and provide a @@ -95,8 +94,8 @@ import java.util.Map; *

    *
  1. It identifies why the developer is creating the exception and who they believe can take action on it. * This context allows for code reviewers and other developers to evaluate the message with the persona in mind
  2. - *
  3. It can be used as a way to control which error messages should be routed where. For example, a user-targetted - * error message should be able to be exposed directly to the user, while an operator-targetted error message should + *
  4. It can be used as a way to control which error messages should be routed where. For example, a user-targeted + * error message should be able to be exposed directly to the user, while an operator-targeted error message should * perhaps be routed to the operators of the system instead of the end user firing a query.
  5. *
*

@@ -105,11 +104,11 @@ import java.util.Map; *

* The error code is a code that indicates a grouping of error messages. There is no forced structure around whether * a specific error code can be reused for different problems or not. That is, an error code like "general" will get - * reused in many different places as it's the basic error code used whenever a DruidException is created in-line. But, + * reused in many places as it's the basic error code used whenever a DruidException is created in-line. But, * we might decide that a specific type of error should be identified explicitly by its error code and should only mean * one thing, in which case that error code might only exist on a single error. *

- * The error message is a message written targetting the target persona. It should have values interpolated into it + * The error message is a message written targeting the target persona. It should have values interpolated into it * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information. *

* The context is a place to add extra information about the error that is not necessarily interpolated into the @@ -132,7 +131,7 @@ import java.util.Map; public class DruidException extends RuntimeException { /** - * Starts building an "general" DruidException targetting the specific persona. + * Starts building a "general" DruidException targeting the specific persona. * * @param persona the target persona of the exception message * @return a builder that can be used to complete the creation of the DruidException @@ -266,7 +265,7 @@ public class DruidException extends RuntimeException } /** - * The persona that the message on a DruidException is targetting + * The persona that the message on a DruidException is targeting */ public enum Persona { @@ -307,7 +306,7 @@ public class DruidException extends RuntimeException DEFENSIVE(500), /** * Means that the input provided was malformed in some way. Generally speaking, it is hoped that errors of this - * category have messages written either targetting the USER or ADMIN personas as those are the general users + * category have messages written either targeting the USER or ADMIN personas as those are the general users * of the APIs who could generate invalid inputs. */ INVALID_INPUT(400), diff --git a/processing/src/main/java/org/apache/druid/query/ExecutionMode.java b/processing/src/main/java/org/apache/druid/query/ExecutionMode.java new file mode 100644 index 00000000000..9809d66051b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/ExecutionMode.java @@ -0,0 +1,35 @@ +/* + * 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.druid.query; + +public enum ExecutionMode +{ + + /** + * This mode executes the query in a blocking way. The results are returned as part of the original post query call. Current sql/native endpoints are sync execution. + */ + SYNC, + + /** + * This mode executes the query in a non-blocking way. The results are returned as part of subsequent get results call. Currently, the msq engine uses this mode of execution. + */ + ASYNC + +} diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index ea21987bd18..203e63f23f6 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -88,6 +88,9 @@ public class QueryContexts public static final String CTX_SQL_QUERY_ID = BaseQuery.SQL_QUERY_ID; public static final String CTX_SQL_STRINGIFY_ARRAYS = "sqlStringifyArrays"; + // SQL statement resource specific keys + public static final String CTX_EXECUTION_MODE = "executionMode"; + // Defaults public static final boolean DEFAULT_BY_SEGMENT = false; public static final boolean DEFAULT_POPULATE_CACHE = true; @@ -425,8 +428,20 @@ public class QueryContexts public static > E getAsEnum(String key, Object value, Class clazz, E defaultValue) { - if (value == null) { + E result = getAsEnum(key, value, clazz); + if (result == null) { return defaultValue; + } else { + return result; + } + } + + + @Nullable + public static > E getAsEnum(String key, Object value, Class clazz) + { + if (value == null) { + return null; } try { diff --git a/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java b/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java index d47bb558fe9..38b5384ded9 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java @@ -295,4 +295,26 @@ public class QueryContextsTest () -> query.context().getEnum("e2", QueryContexts.Vectorize.class, QueryContexts.Vectorize.FALSE) ); } + + @Test + public void testExecutionModeEnum() + { + Query query = new TestQuery( + new TableDataSource("test"), + new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), + false, + ImmutableMap.of(QueryContexts.CTX_EXECUTION_MODE, "SYNC", QueryContexts.CTX_EXECUTION_MODE + "_1", "ASYNC") + ); + + Assert.assertEquals( + ExecutionMode.SYNC, + query.context().getEnum(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.class, ExecutionMode.ASYNC) + ); + + Assert.assertEquals( + ExecutionMode.ASYNC, + query.context().getEnum(QueryContexts.CTX_EXECUTION_MODE + "_1", ExecutionMode.class, ExecutionMode.SYNC) + ); + } + } diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java index 85344e8cdbd..dac5dfc2263 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -20,6 +20,7 @@ package org.apache.druid.rpc.indexing; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.rpc.ServiceRetryPolicy; @@ -52,5 +53,7 @@ public interface OverlordClient ListenableFuture> taskReportAsMap(String taskId); + ListenableFuture taskPayload(String taskId); + OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy); } diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java index 4b9cfd7433f..5c28d87a8d1 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.indexer.TaskStatus; @@ -131,6 +132,22 @@ public class OverlordClientImpl implements OverlordClient ); } + @Override + public ListenableFuture taskPayload(String taskId) + { + final String path = StringUtils.format("/druid/indexer/v1/task/%s", StringUtils.urlEncode(taskId)); + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> deserialize(holder, new TypeReference() + { + }) + ); + } + @Override public OverlordClientImpl withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java index 907973edcf7..dbfa1ec1047 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java @@ -59,6 +59,12 @@ public class NoopOverlordClient implements OverlordClient throw new UnsupportedOperationException(); } + @Override + public ListenableFuture taskPayload(String taskId) + { + throw new UnsupportedOperationException(); + } + @Override public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java new file mode 100644 index 00000000000..26fb972ac5c --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -0,0 +1,58 @@ +/* + * 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.druid.rpc.indexing; + +import com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery; +import org.apache.druid.client.indexing.ClientTaskQuery; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.rpc.MockServiceClient; +import org.apache.druid.rpc.RequestBuilder; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.concurrent.ExecutionException; + +public class OverlordClientImplTest +{ + + + @Test + public void testTaskPayload() throws ExecutionException, InterruptedException, JsonProcessingException + { + final String taskID = "taskId_1"; + MockServiceClient client = new MockServiceClient(); + final OverlordClientImpl overlordClient = new OverlordClientImpl(client, DefaultObjectMapper.INSTANCE); + + ClientTaskQuery clientTaskQuery = new ClientKillUnusedSegmentsTaskQuery(taskID, "test", null, null); + + client.expect(new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/task/" + taskID), + HttpResponseStatus.OK, + Collections.emptyMap(), + DefaultObjectMapper.INSTANCE.writeValueAsBytes(new TaskPayloadResponse(taskID, clientTaskQuery)) + ); + + Assert.assertEquals(clientTaskQuery, overlordClient.taskPayload(taskID).get().getPayload()); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/DirectStatement.java b/sql/src/main/java/org/apache/druid/sql/DirectStatement.java index 0bcf0f684ca..21ebe9e17ba 100644 --- a/sql/src/main/java/org/apache/druid/sql/DirectStatement.java +++ b/sql/src/main/java/org/apache/druid/sql/DirectStatement.java @@ -230,8 +230,8 @@ public class DirectStatement extends AbstractStatement implements Cancelable catch (RelOptPlanner.CannotPlanException e) { // Not sure if this is even thrown here. throw DruidException.forPersona(DruidException.Persona.DEVELOPER) - .ofCategory(DruidException.Category.UNCATEGORIZED) - .build(e, "Problem planning SQL query"); + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Problem planning SQL query"); } catch (RuntimeException e) { state = State.FAILED; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 8bffc909efa..0dfec3759e0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -135,6 +135,40 @@ public class CalciteTests }; } }; + + public static final AuthorizerMapper TEST_EXTERNAL_AUTHORIZER_MAPPER = new AuthorizerMapper(null) + { + @Override + public Authorizer getAuthorizer(String name) + { + return (authenticationResult, resource, action) -> { + if (TEST_SUPERUSER_NAME.equals(authenticationResult.getIdentity())) { + return Access.OK; + } + + switch (resource.getType()) { + case ResourceType.DATASOURCE: + if (FORBIDDEN_DATASOURCE.equals(resource.getName())) { + return new Access(false); + } else { + return Access.OK; + } + case ResourceType.VIEW: + if ("forbiddenView".equals(resource.getName())) { + return new Access(false); + } else { + return Access.OK; + } + case ResourceType.QUERY_CONTEXT: + case ResourceType.EXTERNAL: + return Access.OK; + default: + return new Access(false); + } + }; + } + }; + public static final AuthenticatorMapper TEST_AUTHENTICATOR_MAPPER; static { diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 7dbc5ce6931..d92b1cbcd44 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -2056,7 +2056,7 @@ public class SqlResourceTest extends CalciteTestBase return JSON_MAPPER.readValue(responseToByteArray(resp), clazz); } - private byte[] responseToByteArray(Response resp) throws IOException + public static byte[] responseToByteArray(Response resp) throws IOException { if (resp.getEntity() instanceof StreamingOutput) { ByteArrayOutputStream baos = new ByteArrayOutputStream(); From 233233c92d906372cff3144e3d03191ab45f32f4 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Wed, 28 Jun 2023 17:54:24 +0530 Subject: [PATCH 44/74] Add query context parameter to control limiting select rows (#14476) * Add query context parameter to control limiting select rows * Add unit tests * Address review comments * Address review comments * Address review comments --- .../apache/druid/msq/exec/ControllerImpl.java | 9 ++- .../msq/indexing/MSQSelectDestination.java | 47 +++++++++++++ .../msq/indexing/report/MSQResultsReport.java | 45 +++++++----- .../msq/util/MultiStageQueryContext.java | 17 +++++ .../apache/druid/msq/exec/MSQSelectTest.java | 69 +++++++++++++++++++ .../indexing/report/MSQTaskReportTest.java | 9 ++- .../apache/druid/msq/test/MSQTestBase.java | 16 ++++- .../msq/util/MultiStageQueryContextTest.java | 7 ++ .../testing/utils/MsqTestQueryHelper.java | 7 +- 9 files changed, 203 insertions(+), 23 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSelectDestination.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index db5e6997179..72657ed0e28 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -90,6 +90,7 @@ import org.apache.druid.msq.indexing.DataSourceMSQDestination; import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.indexing.InputChannelsImpl; import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQSelectDestination; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; @@ -467,7 +468,8 @@ public class ControllerImpl implements Controller queryDef, resultsYielder, task.getQuerySpec().getColumnMappings(), - task.getSqlTypeNames() + task.getSqlTypeNames(), + MultiStageQueryContext.getSelectDestination(task.getQuerySpec().getQuery().context()) ); } else { resultsReport = null; @@ -2032,7 +2034,8 @@ public class ControllerImpl implements Controller final QueryDefinition queryDef, final Yielder resultsYielder, final ColumnMappings columnMappings, - @Nullable final List sqlTypeNames + @Nullable final List sqlTypeNames, + final MSQSelectDestination selectDestination ) { final RowSignature querySignature = queryDef.getFinalStageDefinition().getSignature(); @@ -2047,7 +2050,7 @@ public class ControllerImpl implements Controller ); } - return new MSQResultsReport(mappedSignature.build(), sqlTypeNames, resultsYielder, null); + return MSQResultsReport.createReportAndLimitRowsIfNeeded(mappedSignature.build(), sqlTypeNames, resultsYielder, selectDestination); } private static MSQStatusReport makeStatusReport( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSelectDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSelectDestination.java new file mode 100644 index 00000000000..9ef8fe2c1cf --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQSelectDestination.java @@ -0,0 +1,47 @@ +/* + * 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.druid.msq.indexing; + +/** + * Determines the destination for results of select queries. + */ +public enum MSQSelectDestination +{ + /** + * Writes all the results directly to the report. + */ + TASK_REPORT(false), + /** + * Writes the results as frame files to durable storage. Task report can be truncated to a preview. + */ + DURABLE_STORAGE(true); + + private final boolean shouldTruncateResultsInTaskReport; + + public boolean shouldTruncateResultsInTaskReport() + { + return shouldTruncateResultsInTaskReport; + } + + MSQSelectDestination(boolean shouldTruncateResultsInTaskReport) + { + this.shouldTruncateResultsInTaskReport = shouldTruncateResultsInTaskReport; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQResultsReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQResultsReport.java index 56359f8cdf9..fd06d1d79f7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQResultsReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/report/MSQResultsReport.java @@ -24,10 +24,12 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.common.config.Configs; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.msq.exec.Limits; +import org.apache.druid.msq.indexing.MSQSelectDestination; import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; @@ -44,30 +46,20 @@ public class MSQResultsReport private final List signature; @Nullable private final List sqlTypeNames; - private final List results; + private final Yielder resultYielder; private final boolean resultsTruncated; public MSQResultsReport( final List signature, @Nullable final List sqlTypeNames, - Yielder resultYielder, + final Yielder resultYielder, @Nullable Boolean resultsTruncated ) { this.signature = Preconditions.checkNotNull(signature, "signature"); this.sqlTypeNames = sqlTypeNames; - this.results = new ArrayList<>(); - int rowCount = 0; - while (!resultYielder.isDone() && rowCount < Limits.MAX_SELECT_RESULT_ROWS) { - results.add(resultYielder.get()); - resultYielder = resultYielder.next(null); - ++rowCount; - } - if (resultsTruncated != null) { - this.resultsTruncated = !resultYielder.isDone() || resultsTruncated; - } else { - this.resultsTruncated = !resultYielder.isDone(); - } + this.resultYielder = Preconditions.checkNotNull(resultYielder, "resultYielder"); + this.resultsTruncated = Configs.valueOrDefault(resultsTruncated, false); } /** @@ -84,6 +76,27 @@ public class MSQResultsReport return new MSQResultsReport(signature, sqlTypeNames, Yielders.each(Sequences.simple(results)), resultsTruncated); } + public static MSQResultsReport createReportAndLimitRowsIfNeeded( + final List signature, + @Nullable final List sqlTypeNames, + Yielder resultYielder, + MSQSelectDestination selectDestination + ) + { + if (selectDestination.shouldTruncateResultsInTaskReport()) { + List results = new ArrayList<>(); + int rowCount = 0; + while (!resultYielder.isDone() && rowCount < Limits.MAX_SELECT_RESULT_ROWS) { + results.add(resultYielder.get()); + resultYielder = resultYielder.next(null); + ++rowCount; + } + return new MSQResultsReport(signature, sqlTypeNames, Yielders.each(Sequences.simple(results)), !resultYielder.isDone()); + } else { + return new MSQResultsReport(signature, sqlTypeNames, resultYielder, false); + } + } + @JsonProperty("signature") public List getSignature() { @@ -99,9 +112,9 @@ public class MSQResultsReport } @JsonProperty("results") - public List getResults() + public Yielder getResultYielder() { - return results; + return resultYielder; } @JsonProperty("resultsTruncted") diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 02f5f7550d7..3c951c7d0cc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -27,6 +27,7 @@ import com.opencsv.RFC4180Parser; import com.opencsv.RFC4180ParserBuilder; import org.apache.druid.msq.exec.ClusterStatisticsMergeMode; import org.apache.druid.msq.exec.Limits; +import org.apache.druid.msq.indexing.MSQSelectDestination; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.sql.MSQMode; import org.apache.druid.query.QueryContext; @@ -64,6 +65,10 @@ import java.util.stream.Collectors; * Can be PARALLEL, SEQUENTIAL or AUTO. See {@link ClusterStatisticsMergeMode} for more information on each mode. * Default value is SEQUENTIAL * + *

  • selectDestination: If the query is a Select, determines the location to write results to, once the query + * is finished. Depending on the location, the results might also be truncated to {@link Limits#MAX_SELECT_RESULT_ROWS}. + * Default value is {@link MSQSelectDestination#TASK_REPORT}, which writes all the results to the report. + * *
  • useAutoColumnSchemas: Temporary flag to allow experimentation using * {@link org.apache.druid.segment.AutoTypeColumnSchema} for all 'standard' type columns during segment generation, * see {@link DimensionSchemaUtils#createDimensionSchema} for more details. @@ -87,6 +92,8 @@ public class MultiStageQueryContext public static final String CTX_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; private static final boolean DEFAULT_DURABLE_SHUFFLE_STORAGE = false; + public static final String CTX_SELECT_DESTINATION = "selectDestination"; + private static final String DEFAULT_SELECT_DESTINATION = MSQSelectDestination.TASK_REPORT.toString(); public static final String CTX_FAULT_TOLERANCE = "faultTolerance"; public static final boolean DEFAULT_FAULT_TOLERANCE = false; @@ -204,6 +211,16 @@ public class MultiStageQueryContext ); } + public static MSQSelectDestination getSelectDestination(final QueryContext queryContext) + { + return MSQSelectDestination.valueOf( + queryContext.getString( + CTX_SELECT_DESTINATION, + DEFAULT_SELECT_DESTINATION + ) + ); + } + public static int getRowsInMemory(final QueryContext queryContext) { return queryContext.getInt(CTX_ROWS_IN_MEMORY, DEFAULT_ROWS_IN_MEMORY); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 5e628fc3dbd..0f4210e7f59 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -35,12 +35,14 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.msq.indexing.MSQSelectDestination; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.test.CounterSnapshotMatcher; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestFileUtils; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.LookupDataSource; import org.apache.druid.query.QueryDataSource; @@ -74,6 +76,7 @@ import org.apache.druid.sql.calcite.planner.JoinAlgorithm; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.util.CalciteTests; import org.hamcrest.CoreMatchers; +import org.junit.Assert; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.runner.RunWith; @@ -88,6 +91,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -1852,6 +1856,71 @@ public class MSQSelectTest extends MSQTestBase result.add(new Object[]{1}); } + Map queryContext = new HashMap<>(context); + queryContext.put(MultiStageQueryContext.CTX_SELECT_DESTINATION, MSQSelectDestination.DURABLE_STORAGE.toString()); + + testSelectQuery() + .setSql(StringUtils.format( + " SELECT 1 as \"timestamp\"\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [%s],\"type\":\"local\"}',\n" + + " '{\"type\": \"csv\", \"hasHeaderRow\": true}',\n" + + " '[{\"name\": \"timestamp\", \"type\": \"string\"}]'\n" + + " )\n" + + ")", + externalFiles + )) + .setExpectedRowSignature(dummyRowSignature) + .setExpectedMSQSpec( + MSQSpec + .builder() + .query(newScanQueryBuilder() + .dataSource(new ExternalDataSource( + new LocalInputSource(null, null, Collections.nCopies(numFiles, toRead)), + new CsvInputFormat(null, null, null, true, 0), + RowSignature.builder().add("timestamp", ColumnType.STRING).build() + )) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("v0") + .virtualColumns(new ExpressionVirtualColumn("v0", ExprEval.of(1L).toExpr(), ColumnType.LONG)) + .context(defaultScanQueryContext( + queryContext, + RowSignature.builder().add("v0", ColumnType.LONG).build() + )) + .build() + ) + .columnMappings(new ColumnMappings( + ImmutableList.of( + new ColumnMapping("v0", "timestamp") + ) + )) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build()) + .setQueryContext(queryContext) + .setExpectedResultRows(result) + .verifyResults(); + } + + @Test + public void testSelectRowsGetUntruncatedInReportsByDefault() throws IOException + { + RowSignature dummyRowSignature = RowSignature.builder().add("timestamp", ColumnType.LONG).build(); + + final int numFiles = 200; + + final File toRead = MSQTestFileUtils.getResourceAsTemporaryFile(temporaryFolder, this, "/wikipedia-sampled.json"); + final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(toRead.getAbsolutePath()); + + String externalFiles = String.join(", ", Collections.nCopies(numFiles, toReadFileNameAsJson)); + + List result = new ArrayList<>(); + for (int i = 0; i < 3800; ++i) { + result.add(new Object[]{1}); + } + + Assert.assertTrue(result.size() > Limits.MAX_SELECT_RESULT_ROWS); + testSelectQuery() .setSql(StringUtils.format( " SELECT 1 as \"timestamp\"\n" diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index b226583922d..eeac9486dc9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -31,6 +31,7 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.guice.MSQIndexingModule; @@ -50,6 +51,7 @@ import org.junit.rules.TemporaryFolder; import java.io.File; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -124,8 +126,13 @@ public class MSQTaskReportTest Assert.assertEquals(report.getPayload().getStatus().getPendingTasks(), report2.getPayload().getStatus().getPendingTasks()); Assert.assertEquals(report.getPayload().getStages(), report2.getPayload().getStages()); - final List results2 = report2.getPayload().getResults().getResults(); + Yielder yielder = report2.getPayload().getResults().getResultYielder(); + final List results2 = new ArrayList<>(); + while (!yielder.isDone()) { + results2.add(yielder.get()); + yielder = yielder.next(null); + } Assert.assertEquals(results.size(), results2.size()); for (int i = 0; i < results.size(); i++) { Assert.assertArrayEquals(results.get(i), results2.get(i)); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 0138b957b32..772279ada79 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -69,6 +69,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.input.InputSourceModule; @@ -768,7 +769,20 @@ public class MSQTestBase extends BaseCalciteQueryTest if (resultsReport == null) { return null; } else { - return resultsReport.getResults(); + Yielder yielder = resultsReport.getResultYielder(); + List rows = new ArrayList<>(); + while (!yielder.isDone()) { + rows.add(yielder.get()); + yielder = yielder.next(null); + } + try { + yielder.close(); + } + catch (IOException e) { + throw new ISE("Unable to get results from the report"); + } + + return rows; } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java index bda5d845304..567e6d8a1f7 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java @@ -22,6 +22,7 @@ package org.apache.druid.msq.util; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.msq.indexing.MSQSelectDestination; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.query.BadQueryContextException; import org.apache.druid.query.QueryContext; @@ -263,6 +264,12 @@ public class MultiStageQueryContextTest Assert.assertEquals("nonStrict", MultiStageQueryContext.getMSQMode(QueryContext.of(propertyMap))); } + @Test + public void limitSelectResultReturnsDefaultValue() + { + Assert.assertEquals(MSQSelectDestination.TASK_REPORT, MultiStageQueryContext.getSelectDestination(QueryContext.empty())); + } + @Test public void testUseAutoSchemas() { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java index 5e1a40df073..7525cb9d874 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java @@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; import org.apache.druid.msq.indexing.report.MSQResultsReport; import org.apache.druid.msq.indexing.report.MSQTaskReport; @@ -200,15 +201,17 @@ public class MsqTestQueryHelper extends AbstractTestQueryHelper> actualResults = new ArrayList<>(); - List results = resultsReport.getResults(); + Yielder yielder = resultsReport.getResultYielder(); List rowSignature = resultsReport.getSignature(); - for (Object[] row : results) { + while (!yielder.isDone()) { + Object[] row = yielder.get(); Map rowWithFieldNames = new LinkedHashMap<>(); for (int i = 0; i < row.length; ++i) { rowWithFieldNames.put(rowSignature.get(i).getName(), row[i]); } actualResults.add(rowWithFieldNames); + yielder = yielder.next(null); } QueryResultVerifier.ResultVerificationObject resultsComparison = QueryResultVerifier.compareResults( From 82fbb31c7c55f01629191e5072683f0fdb87c7db Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 28 Jun 2023 10:30:27 -0700 Subject: [PATCH 45/74] Properly read SQL-compatible segments in default-value mode. (#14142) * Properly read SQL-compatible segments in default-value mode. Main changes: 1) Dictionary-encoded and front-coded string columns: in default-value mode, detect cases where a dictionary has the empty string in it, then either combine it with null (if null is present) or replace it with null (if null is not present). 2) Numeric nullable columns: in default-value mode, ignore the null value bitmap. This causes all null numbers to be read as zeroes. Testing strategy: 1) Add a mmappedWithSqlCompatibleNulls case to BaseFilterTest that writes segments under SQL-compatible mode, and reads them under default-value mode. 2) Unit tests for the new wrapper classes (CombineFirstTwoEntriesIndexed, CombineFirstTwoValuesColumnarInts, CombineFirstTwoValuesColumnarMultiInts, CombineFirstTwoValuesIndexedInts). * Fix a mistake, use more singlethreadedness. * WIP * Tests, improvements. * Style. * See Spot bug. * Remove unused method. * Address review comments. 1) Read bitmaps even if we don't retain them. 2) Combine StringFrontCodedDictionaryEncodedColumn and ScalarStringDictionaryEncodedColumn. * Add missing tests. --- .../druid/common/config/NullHandling.java | 43 ++ .../column/StringDictionaryEncodedColumn.java | 20 +- ...=> StringUtf8DictionaryEncodedColumn.java} | 40 +- .../druid/segment/data/CachingIndexed.java | 14 +- .../druid/segment/data/GenericIndexed.java | 13 - .../nested/NestedCommonFormatColumn.java | 4 +- .../ScalarStringColumnAndIndexSupplier.java | 6 +- .../nested/ScalarStringColumnSerializer.java | 7 +- .../ScalarStringDictionaryEncodedColumn.java | 382 ------------------ .../serde/CombineFirstTwoEntriesIndexed.java | 196 +++++++++ .../CombineFirstTwoValuesColumnarInts.java | 48 +++ ...ombineFirstTwoValuesColumnarMultiInts.java | 106 +++++ .../CombineFirstTwoValuesIndexedInts.java | 97 +++++ .../DictionaryEncodedColumnSupplier.java | 44 +- .../DictionaryEncodedStringIndexSupplier.java | 28 +- .../serde/DoubleNumericColumnPartSerdeV2.java | 10 +- .../serde/FloatNumericColumnPartSerdeV2.java | 12 +- .../serde/LongNumericColumnPartSerdeV2.java | 12 +- .../ReplaceFirstValueWithNullIndexed.java | 131 ++++++ .../StringFrontCodedColumnIndexSupplier.java | 31 +- ...tCodedDictionaryEncodedColumnSupplier.java | 31 +- .../druid/common/config/NullHandlingTest.java | 86 ++++ .../apache/druid/segment/IndexBuilder.java | 40 +- .../segment/IndexMergerNullHandlingTest.java | 8 +- .../druid/segment/filter/BaseFilterTest.java | 104 +++-- .../ScalarStringColumnSupplierTest.java | 7 +- .../CombineFirstTwoEntriesIndexedTest.java | 162 ++++++++ ...CombineFirstTwoValuesColumnarIntsTest.java | 47 +++ ...neFirstTwoValuesColumnarMultiIntsTest.java | 119 ++++++ .../CombineFirstTwoValuesIndexedIntsTest.java | 101 +++++ .../ReplaceFirstValueWithNullIndexedTest.java | 137 +++++++ 31 files changed, 1553 insertions(+), 533 deletions(-) rename processing/src/main/java/org/apache/druid/segment/column/{StringFrontCodedDictionaryEncodedColumn.java => StringUtf8DictionaryEncodedColumn.java} (90%) delete mode 100644 processing/src/main/java/org/apache/druid/segment/nested/ScalarStringDictionaryEncodedColumn.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoEntriesIndexed.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarInts.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarMultiInts.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesIndexedInts.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/ReplaceFirstValueWithNullIndexed.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoEntriesIndexedTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarIntsTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarMultiIntsTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesIndexedIntsTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/ReplaceFirstValueWithNullIndexedTest.java diff --git a/processing/src/main/java/org/apache/druid/common/config/NullHandling.java b/processing/src/main/java/org/apache/druid/common/config/NullHandling.java index f7d31469c31..8e6f242e9a3 100644 --- a/processing/src/main/java/org/apache/druid/common/config/NullHandling.java +++ b/processing/src/main/java/org/apache/druid/common/config/NullHandling.java @@ -22,8 +22,10 @@ package org.apache.druid.common.config; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import com.google.inject.Inject; +import org.apache.druid.segment.data.Indexed; import javax.annotation.Nullable; +import java.nio.ByteBuffer; /** * Helper class for NullHandling. This class is used to switch between SQL compatible Null Handling behavior @@ -163,4 +165,45 @@ public class NullHandling { return replaceWithDefault() ? Strings.isNullOrEmpty(value) : value == null; } + + public static boolean isNullOrEquivalent(@Nullable ByteBuffer buffer) + { + return buffer == null || (replaceWithDefault() && buffer.remaining() == 0); + } + + /** + * Given a UTF-8 dictionary, returns whether the first two entries must be coalesced into a single null entry. + * This happens if we are in default-value mode and the first two entries are null and empty string. + * + * This and {@link #mustReplaceFirstValueWithNullInDictionary(Indexed)} are never both true. + * + * Provided to enable compatibility for segments written under {@link #sqlCompatible()} mode but + * read under {@link #replaceWithDefault()} mode. + */ + public static boolean mustCombineNullAndEmptyInDictionary(final Indexed dictionaryUtf8) + { + return NullHandling.replaceWithDefault() + && dictionaryUtf8.size() >= 2 + && isNullOrEquivalent(dictionaryUtf8.get(0)) + && isNullOrEquivalent(dictionaryUtf8.get(1)); + } + + /** + * Given a UTF-8 dictionary, returns whether the first entry must be replaced with null. This happens if we + * are in default-value mode and the first entry is an empty string. (Default-value mode expects it to be null.) + * + * This and {@link #mustCombineNullAndEmptyInDictionary(Indexed)} are never both true. + * + * Provided to enable compatibility for segments written under {@link #sqlCompatible()} mode but + * read under {@link #replaceWithDefault()} mode. + */ + public static boolean mustReplaceFirstValueWithNullInDictionary(final Indexed dictionaryUtf8) + { + if (NullHandling.replaceWithDefault() && dictionaryUtf8.size() >= 1) { + final ByteBuffer firstValue = dictionaryUtf8.get(0); + return firstValue != null && firstValue.remaining() == 0; + } + + return false; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java index 66bbd71c5dd..69b3af4c140 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java @@ -27,7 +27,6 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractDimensionSelector; import org.apache.druid.segment.DimensionSelectorUtils; import org.apache.druid.segment.IdLookup; -import org.apache.druid.segment.data.CachingIndexed; import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.ColumnarMultiInts; import org.apache.druid.segment.data.Indexed; @@ -45,6 +44,7 @@ import org.apache.druid.segment.vector.VectorObjectSelector; import org.apache.druid.utils.CloseableUtils; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -60,19 +60,19 @@ public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn cachedDictionary; + private final Indexed dictionary; private final Indexed dictionaryUtf8; public StringDictionaryEncodedColumn( @Nullable ColumnarInts singleValueColumn, @Nullable ColumnarMultiInts multiValueColumn, - CachingIndexed dictionary, + Indexed dictionary, Indexed dictionaryUtf8 ) { this.column = singleValueColumn; this.multiValueColumn = multiValueColumn; - this.cachedDictionary = dictionary; + this.dictionary = dictionary; this.dictionaryUtf8 = dictionaryUtf8; } @@ -104,7 +104,7 @@ public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn} for a column which uses a {@link FrontCodedIndexed} to store its value - * dictionary, which 'delta encodes' strings (instead of {@link org.apache.druid.segment.data.GenericIndexed} like - * {@link StringDictionaryEncodedColumn}). + * {@link DictionaryEncodedColumn} for a column which has only a UTF-8 dictionary, no String dictionary. *

    - * This class is otherwise nearly identical to {@link StringDictionaryEncodedColumn} other than the dictionary - * difference. + * This class is otherwise nearly identical to {@link StringDictionaryEncodedColumn} other than lacking a + * String dictionary. *

    * Implements {@link NestedCommonFormatColumn} so it can be used as a reader for single value string specializations * of {@link org.apache.druid.segment.AutoTypeColumnIndexer}. */ -public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncodedColumn, +public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColumn, NestedCommonFormatColumn { @Nullable private final ColumnarInts column; @Nullable private final ColumnarMultiInts multiValueColumn; - private final FrontCodedIndexed utf8Dictionary; + private final Indexed utf8Dictionary; - public StringFrontCodedDictionaryEncodedColumn( + public StringUtf8DictionaryEncodedColumn( @Nullable ColumnarInts singleValueColumn, @Nullable ColumnarMultiInts multiValueColumn, - FrontCodedIndexed utf8Dictionary + Indexed utf8Dictionary ) { this.column = singleValueColumn; @@ -102,6 +99,9 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public IndexedInts getMultiValueRow(int rowNum) { + if (!hasMultipleValues()) { + throw new UnsupportedOperationException("Column is not multi-valued"); + } return multiValueColumn.get(rowNum); } @@ -154,7 +154,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public String lookupName(int id) { - final String value = StringFrontCodedDictionaryEncodedColumn.this.lookupName(id); + final String value = StringUtf8DictionaryEncodedColumn.this.lookupName(id); return extractionFn == null ? value : extractionFn.apply(value); } @@ -190,7 +190,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode if (extractionFn != null) { throw new UnsupportedOperationException("cannot perform lookup when applying an extraction function"); } - return StringFrontCodedDictionaryEncodedColumn.this.lookupId(name); + return StringUtf8DictionaryEncodedColumn.this.lookupId(name); } } @@ -291,7 +291,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - inspector.visit("column", StringFrontCodedDictionaryEncodedColumn.this); + inspector.visit("column", StringUtf8DictionaryEncodedColumn.this); } }; } else { @@ -332,7 +332,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - inspector.visit("column", StringFrontCodedDictionaryEncodedColumn.this); + inspector.visit("column", StringUtf8DictionaryEncodedColumn.this); } }; } @@ -381,7 +381,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public String lookupName(final int id) { - return StringFrontCodedDictionaryEncodedColumn.this.lookupName(id); + return StringUtf8DictionaryEncodedColumn.this.lookupName(id); } @Nullable @@ -394,7 +394,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public int lookupId(@Nullable String name) { - return StringFrontCodedDictionaryEncodedColumn.this.lookupId(name); + return StringUtf8DictionaryEncodedColumn.this.lookupId(name); } } @@ -421,7 +421,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public String lookupName(final int id) { - return StringFrontCodedDictionaryEncodedColumn.this.lookupName(id); + return StringUtf8DictionaryEncodedColumn.this.lookupName(id); } @Nullable @@ -435,7 +435,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public int lookupId(@Nullable String name) { - return StringFrontCodedDictionaryEncodedColumn.this.lookupId(name); + return StringUtf8DictionaryEncodedColumn.this.lookupId(name); } } @@ -457,7 +457,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public String lookupName(int id) { - return StringFrontCodedDictionaryEncodedColumn.this.lookupName(id); + return StringUtf8DictionaryEncodedColumn.this.lookupName(id); } } return new StringVectorSelector(); @@ -473,7 +473,7 @@ public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncode @Override public String lookupName(int id) { - return StringFrontCodedDictionaryEncodedColumn.this.lookupName(id); + return StringUtf8DictionaryEncodedColumn.this.lookupName(id); } } return new MultiStringVectorSelector(); diff --git a/processing/src/main/java/org/apache/druid/segment/data/CachingIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/CachingIndexed.java index 4c4823550a7..6632eaa95e6 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/CachingIndexed.java +++ b/processing/src/main/java/org/apache/druid/segment/data/CachingIndexed.java @@ -27,6 +27,7 @@ import javax.annotation.Nullable; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.Map; +import java.util.function.ToIntFunction; public class CachingIndexed implements CloseableIndexed { @@ -34,7 +35,8 @@ public class CachingIndexed implements CloseableIndexed private static final Logger log = new Logger(CachingIndexed.class); - private final GenericIndexed.BufferIndexed delegate; + private final Indexed delegate; + private final ToIntFunction sizeFn; @Nullable private final SizedLRUMap cachedValues; @@ -44,12 +46,14 @@ public class CachingIndexed implements CloseableIndexed * CachingIndexed objects are not thread safe and should only be used by a single thread at a time. * CachingIndexed objects must be closed to release any underlying cache resources. * - * @param delegate the GenericIndexed to wrap with a lookup cache. + * @param delegate the Indexed to wrap with a lookup cache. + * @param sizeFn function that determines the size in bytes of an object * @param lookupCacheSize maximum size in bytes of the lookup cache if greater than zero */ - public CachingIndexed(GenericIndexed delegate, final int lookupCacheSize) + public CachingIndexed(Indexed delegate, final ToIntFunction sizeFn, final int lookupCacheSize) { - this.delegate = delegate.singleThreaded(); + this.delegate = delegate; + this.sizeFn = sizeFn; if (lookupCacheSize > 0) { log.debug("Allocating column cache of max size[%d]", lookupCacheSize); @@ -75,7 +79,7 @@ public class CachingIndexed implements CloseableIndexed } final T value = delegate.get(index); - cachedValues.put(index, value, delegate.getLastValueSize()); + cachedValues.put(index, value, sizeFn.applyAsInt(value)); return value; } else { return delegate.get(index); diff --git a/processing/src/main/java/org/apache/druid/segment/data/GenericIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/GenericIndexed.java index 6f549e9a0a8..11a8b61531b 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/org/apache/druid/segment/data/GenericIndexed.java @@ -459,8 +459,6 @@ public class GenericIndexed implements CloseableIndexed, Serializer */ public abstract class BufferIndexed implements Indexed { - int lastReadSize; - @Override public int size() { @@ -492,7 +490,6 @@ public class GenericIndexed implements CloseableIndexed, Serializer || copyValueBuffer.get(startOffset - Integer.BYTES) == NULL_VALUE_SIZE_MARKER)) { return null; } - lastReadSize = size; // ObjectStrategy.fromByteBuffer() is allowed to reset the limit of the buffer. So if the limit is changed, // position() call could throw an exception, if the position is set beyond the new limit. Calling limit() @@ -511,16 +508,6 @@ public class GenericIndexed implements CloseableIndexed, Serializer @Nullable protected abstract ByteBuffer getByteBuffer(int index); - /** - * This method makes no guarantees with respect to thread safety - * - * @return the size in bytes of the last value read - */ - int getLastValueSize() - { - return lastReadSize; - } - @Override public int indexOf(@Nullable T value) { diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java index 53208409284..59c0070d243 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java @@ -30,6 +30,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; @@ -45,8 +46,7 @@ import java.util.TreeMap; * * @see ScalarDoubleColumn * @see ScalarLongColumn - * @see ScalarStringDictionaryEncodedColumn - * @see org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn + * @see StringUtf8DictionaryEncodedColumn * @see VariantColumn * @see CompressedNestedDataComplexColumn */ diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java index 8c1aa67c998..51b5d2d335d 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java @@ -42,7 +42,7 @@ import org.apache.druid.segment.column.NullValueIndex; import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; import org.apache.druid.segment.column.StringEncodingStrategies; import org.apache.druid.segment.column.StringEncodingStrategy; -import org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.ColumnarInts; @@ -190,13 +190,13 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier(encodedColumnSupplier.get(), stringDictionary.singleThreaded()); + return new StringUtf8DictionaryEncodedColumn(encodedColumnSupplier.get(), null, stringDictionary.singleThreaded()); } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java index 72838cc3d7e..909d0d50f4b 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java @@ -33,6 +33,7 @@ import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.StringEncodingStrategies; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSerializer; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.DictionaryWriter; @@ -45,10 +46,8 @@ import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; /** - * Serializer for a string {@link NestedCommonFormatColumn} that can be read with either - * {@link ScalarStringDictionaryEncodedColumn} or - * {@link org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn} (if written with a front-coded - * dictionary). + * Serializer for a string {@link NestedCommonFormatColumn} that can be read with + * {@link StringUtf8DictionaryEncodedColumn}. */ public class ScalarStringColumnSerializer extends NestedCommonFormatColumnSerializer { diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringDictionaryEncodedColumn.java deleted file mode 100644 index 97d3a1b56d6..00000000000 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringDictionaryEncodedColumn.java +++ /dev/null @@ -1,382 +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.druid.segment.nested; - -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.query.extraction.ExtractionFn; -import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.AbstractDimensionSelector; -import org.apache.druid.segment.IdLookup; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.StringDictionaryEncodedColumn; -import org.apache.druid.segment.column.StringEncodingStrategies; -import org.apache.druid.segment.data.ColumnarInts; -import org.apache.druid.segment.data.Indexed; -import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.data.ReadableOffset; -import org.apache.druid.segment.data.SingleIndexedInt; -import org.apache.druid.segment.filter.BooleanValueMatcher; -import org.apache.druid.segment.historical.HistoricalDimensionSelector; -import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector; -import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; -import org.apache.druid.segment.vector.ReadableVectorOffset; -import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; -import org.apache.druid.segment.vector.VectorObjectSelector; -import org.apache.druid.utils.CloseableUtils; - -import javax.annotation.Nullable; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.BitSet; - -/** - * {@link NestedCommonFormatColumn} specialization for {@link ColumnType#STRING} with a generic buffer based utf8 - * dictionary. This is used when not using the more specific - * {@link org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn}, and only supports single value - * strings. - */ -public class ScalarStringDictionaryEncodedColumn> - implements DictionaryEncodedColumn, NestedCommonFormatColumn -{ - private final ColumnarInts column; - private final TIndexed utf8Dictionary; - - public ScalarStringDictionaryEncodedColumn( - ColumnarInts singleValueColumn, - TIndexed utf8Dictionary - ) - { - this.column = singleValueColumn; - this.utf8Dictionary = utf8Dictionary; - } - - @Override - public int length() - { - return column.size(); - } - - @Override - public boolean hasMultipleValues() - { - return false; - } - - @Override - public int getSingleValueRow(int rowNum) - { - return column.get(rowNum); - } - - @Override - public IndexedInts getMultiValueRow(int rowNum) - { - throw new UnsupportedOperationException("Column is not multi-valued"); - } - - @Override - @Nullable - public String lookupName(int id) - { - final ByteBuffer buffer = utf8Dictionary.get(id); - if (buffer == null) { - return null; - } - return StringUtils.fromUtf8(buffer); - } - - @Override - public int lookupId(String name) - { - return utf8Dictionary.indexOf(StringUtils.toUtf8ByteBuffer(name)); - } - - @Override - public int getCardinality() - { - return utf8Dictionary.size(); - } - - @Override - public HistoricalDimensionSelector makeDimensionSelector( - final ReadableOffset offset, - @Nullable final ExtractionFn extractionFn - ) - { - class SingleValueQueryableDimensionSelector extends AbstractDimensionSelector - implements SingleValueHistoricalDimensionSelector, IdLookup, HistoricalDimensionSelector - { - private final SingleIndexedInt row = new SingleIndexedInt(); - - @Override - public int getValueCardinality() - { - /* - This is technically wrong if - extractionFn != null && (extractionFn.getExtractionType() != ExtractionFn.ExtractionType.ONE_TO_ONE || - !extractionFn.preservesOrdering()) - However current behavior allows some GroupBy-V1 queries to work that wouldn't work otherwise and doesn't - cause any problems due to special handling of extractionFn everywhere. - See https://github.com/apache/druid/pull/8433 - */ - return getCardinality(); - } - - @Override - public String lookupName(int id) - { - final String value = ScalarStringDictionaryEncodedColumn.this.lookupName(id); - return extractionFn == null ? value : extractionFn.apply(value); - } - - @Nullable - @Override - public ByteBuffer lookupNameUtf8(int id) - { - return utf8Dictionary.get(id); - } - - @Override - public boolean supportsLookupNameUtf8() - { - return true; - } - - @Override - public boolean nameLookupPossibleInAdvance() - { - return true; - } - - @Nullable - @Override - public IdLookup idLookup() - { - return extractionFn == null ? this : null; - } - - @Override - public int lookupId(String name) - { - if (extractionFn != null) { - throw new UnsupportedOperationException("cannot perform lookup when applying an extraction function"); - } - return ScalarStringDictionaryEncodedColumn.this.lookupId(name); - } - - @Override - public IndexedInts getRow() - { - row.setValue(getRowValue()); - return row; - } - - public int getRowValue() - { - return column.get(offset.getOffset()); - } - - @Override - public IndexedInts getRow(int offset) - { - row.setValue(getRowValue(offset)); - return row; - } - - @Override - public int getRowValue(int offset) - { - return column.get(offset); - } - - @Override - public ValueMatcher makeValueMatcher(final @Nullable String value) - { - if (extractionFn == null) { - final int valueId = lookupId(value); - if (valueId >= 0) { - return new ValueMatcher() - { - @Override - public boolean matches() - { - return getRowValue() == valueId; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("column", ScalarStringDictionaryEncodedColumn.this); - } - }; - } else { - return BooleanValueMatcher.of(false); - } - } else { - // Employ caching BitSet optimization - return makeValueMatcher(Predicates.equalTo(value)); - } - } - - @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) - { - final BitSet checkedIds = new BitSet(getCardinality()); - final BitSet matchingIds = new BitSet(getCardinality()); - - // Lazy matcher; only check an id if matches() is called. - return new ValueMatcher() - { - @Override - public boolean matches() - { - final int id = getRowValue(); - - if (checkedIds.get(id)) { - return matchingIds.get(id); - } else { - final boolean matches = predicate.apply(lookupName(id)); - checkedIds.set(id); - if (matches) { - matchingIds.set(id); - } - return matches; - } - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("column", ScalarStringDictionaryEncodedColumn.this); - } - }; - } - - @Override - public Object getObject() - { - return lookupName(getRowValue()); - } - - @Override - public Class classOfObject() - { - return String.class; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("column", column); - inspector.visit("offset", offset); - inspector.visit("extractionFn", extractionFn); - } - } - return new SingleValueQueryableDimensionSelector(); - } - - @Override - public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(final ReadableVectorOffset offset) - { - final class StringVectorSelector extends StringDictionaryEncodedColumn.StringSingleValueDimensionVectorSelector - { - public StringVectorSelector() - { - super(column, offset); - } - - @Override - public int getValueCardinality() - { - return getCardinality(); - } - - @Nullable - @Override - public String lookupName(final int id) - { - return ScalarStringDictionaryEncodedColumn.this.lookupName(id); - } - - @Nullable - @Override - public ByteBuffer lookupNameUtf8(int id) - { - return utf8Dictionary.get(id); - } - - @Override - public int lookupId(@Nullable String name) - { - return ScalarStringDictionaryEncodedColumn.this.lookupId(name); - } - } - - return new StringVectorSelector(); - } - - @Override - public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(ReadableVectorOffset vectorOffset) - { - throw new UnsupportedOperationException("Column not multi-valued"); - } - - @Override - public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset) - { - final class StringVectorSelector extends StringDictionaryEncodedColumn.StringVectorObjectSelector - { - public StringVectorSelector() - { - super(column, offset); - } - - @Nullable - @Override - public String lookupName(int id) - { - return ScalarStringDictionaryEncodedColumn.this.lookupName(id); - } - } - return new StringVectorSelector(); - } - - @Override - public void close() throws IOException - { - CloseableUtils.closeAll(column); - } - - @Override - public ColumnType getLogicalType() - { - return ColumnType.STRING; - } - - @Override - public Indexed getStringDictionary() - { - return new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary); - } -} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoEntriesIndexed.java b/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoEntriesIndexed.java new file mode 100644 index 00000000000..3e109946d72 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoEntriesIndexed.java @@ -0,0 +1,196 @@ +/* + * 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.druid.segment.serde; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.data.Indexed; + +import javax.annotation.Nullable; +import java.util.Iterator; +import java.util.Objects; + +/** + * An {@link Indexed} that delegates to an underyling instance, but combines the first two entries. + * + * Unlike {@link CombineFirstTwoValuesIndexedInts}, this class combines the first two *entries*. + * So [0, 1, 2] becomes [(something), 2]. The first two entries, 0 and 1, were replaced with (something). That something + * is given by {@link #newFirstValue()}. + * + * Provided to enable compatibility for segments written under {@link NullHandling#sqlCompatible()} mode but + * read under {@link NullHandling#replaceWithDefault()} mode. + * + * Important note: {@link #isSorted()} returns the same value as the underlying delegate. In this case, this class + * assumes that {@link #newFirstValue()} is the lowest possible value in the universe: including anything in + * {@link #delegate} and anything that might be passed to {@link #indexOf(Object)}. Callers must ensure that this + * precondition is met. + * + * @see NullHandling#mustCombineNullAndEmptyInDictionary(Indexed) + */ +public abstract class CombineFirstTwoEntriesIndexed implements Indexed +{ + private static final int FIRST_ID = 0; + + protected final Indexed delegate; + + protected CombineFirstTwoEntriesIndexed(Indexed delegate) + { + this.delegate = delegate; + + if (delegate.size() < 2) { + throw new ISE("Size[%s] must be >= 2", delegate.size()); + } + } + + /** + * Combine the first two values into a literal null. + */ + public static CombineFirstTwoEntriesIndexed returnNull(final Indexed delegate) + { + return new CombineFirstTwoEntriesIndexed(delegate) + { + @Nullable + @Override + protected T newFirstValue() + { + return null; + } + }; + } + + /** + * Union the first two bitmaps. + */ + public static CombineFirstTwoEntriesIndexed unionBitmaps( + final BitmapFactory bitmapFactory, + final Indexed delegate + ) + { + return new CombineFirstTwoEntriesIndexed(delegate) + { + @Nullable + @Override + protected ImmutableBitmap newFirstValue() + { + return bitmapFactory.union(ImmutableList.of(delegate.get(FIRST_ID), delegate.get(FIRST_ID + 1))); + } + }; + } + + @Nullable + protected abstract T newFirstValue(); + + @Override + public int size() + { + return delegate.size() - 1; + } + + @Nullable + @Override + public T get(int index) + { + if (index == FIRST_ID) { + return newFirstValue(); + } else { + return delegate.get(index + 1); + } + } + + @Override + public int indexOf(@Nullable T value) + { + if (Objects.equals(newFirstValue(), value)) { + return FIRST_ID; + } else { + final int index = delegate.indexOf(value); + + if (index > FIRST_ID + 1) { + // Item found, index needs adjustment. + return index - 1; + } else if (index >= 0) { + // Item found, but shadowed, so really not found. + // Insertion point is after FIRST_ID. (See class-level javadoc: newFirstValue is required to be + // lower than all elements in the universe.) + return -2; + } else if (index >= -2) { + // Item not found, and insertion point is prior to, or within, the shadowed portion of delegate. Return + // insertion point immediately after newFirstValue, since that value is required to be lower than all elements + // in the universe. + return -2; + } else { + // Item not found, and insertion point is after the shadowed portion of delegate. Adjust and return. + return index + 1; + } + } + } + + @Override + public Iterator iterator() + { + final Iterator it = delegate.iterator(); + + // Skip first two values. + //CHECKSTYLE.OFF: Regexp + it.next(); + it.next(); + //CHECKSTYLE.ON: Regexp + + class CoalescingIndexedIterator implements Iterator + { + boolean returnedFirstValue; + + @Override + public boolean hasNext() + { + return !returnedFirstValue || it.hasNext(); + } + + @Override + public T next() + { + if (!returnedFirstValue) { + returnedFirstValue = true; + return newFirstValue(); + } else { + return it.next(); + } + } + } + + return new CoalescingIndexedIterator(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + delegate.inspectRuntimeShape(inspector); + } + + @Override + public boolean isSorted() + { + return delegate.isSorted(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarInts.java b/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarInts.java new file mode 100644 index 00000000000..07ccd303db6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarInts.java @@ -0,0 +1,48 @@ +/* + * 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.druid.segment.serde; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.segment.data.ColumnarInts; +import org.apache.druid.segment.data.Indexed; + +import java.io.IOException; + +/** + * A {@link ColumnarInts} facade over {@link CombineFirstTwoValuesIndexedInts}. + * + * Provided to enable compatibility for segments written under {@link NullHandling#sqlCompatible()} mode but + * read under {@link NullHandling#replaceWithDefault()} mode. + * + * @see NullHandling#mustCombineNullAndEmptyInDictionary(Indexed) + */ +public class CombineFirstTwoValuesColumnarInts extends CombineFirstTwoValuesIndexedInts implements ColumnarInts +{ + public CombineFirstTwoValuesColumnarInts(ColumnarInts delegate) + { + super(delegate); + } + + @Override + public void close() throws IOException + { + ((ColumnarInts) delegate).close(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarMultiInts.java b/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarMultiInts.java new file mode 100644 index 00000000000..4d8cc5f61f6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarMultiInts.java @@ -0,0 +1,106 @@ +/* + * 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.druid.segment.serde; + +import com.google.common.collect.Iterators; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.data.ColumnarMultiInts; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.data.ZeroIndexedInts; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Iterator; + +/** + * A {@link ColumnarMultiInts} that delegates to an underyling instance, but applies + * {@link CombineFirstTwoValuesIndexedInts} to each row's set of values. + * + * Provided to enable compatibility for segments written under {@link NullHandling#sqlCompatible()} mode but + * read under {@link NullHandling#replaceWithDefault()} mode. + * + * @see NullHandling#mustCombineNullAndEmptyInDictionary(Indexed) + */ +public class CombineFirstTwoValuesColumnarMultiInts implements ColumnarMultiInts +{ + private final ColumnarMultiInts delegate; + private final CombineFirstTwoValuesIndexedInts rowValues; + + public CombineFirstTwoValuesColumnarMultiInts(ColumnarMultiInts delegate) + { + this.delegate = delegate; + this.rowValues = new CombineFirstTwoValuesIndexedInts(ZeroIndexedInts.instance()); + } + + @Override + public IndexedInts get(int index) + { + rowValues.delegate = delegate.get(index); + return rowValues; + } + + @Override + public IndexedInts getUnshared(int index) + { + return new CombineFirstTwoValuesIndexedInts(delegate.getUnshared(index)); + } + + @Override + public int size() + { + return delegate.size(); + } + + @Override + public int indexOf(@Nullable IndexedInts value) + { + // No ColumnarMultiInts implement this method + throw new UnsupportedOperationException("Reverse lookup not allowed."); + } + + @Override + public boolean isSorted() + { + return delegate.isSorted(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + delegate.inspectRuntimeShape(inspector); + } + + @Override + public Iterator iterator() + { + return Iterators.transform( + delegate.iterator(), + CombineFirstTwoValuesIndexedInts::new + ); + } + + @Override + public void close() throws IOException + { + delegate.close(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesIndexedInts.java b/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesIndexedInts.java new file mode 100644 index 00000000000..e2e6bf6d6e9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/CombineFirstTwoValuesIndexedInts.java @@ -0,0 +1,97 @@ +/* + * 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.druid.segment.serde; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.IndexedInts; + +/** + * A {@link IndexedInts} that delegates to an underyling instance, but combines the values 0 and 1 into 0, and shifts + * all other values down by one. For example: + * + * - [2, 0, 1] => [1, 0, 0] + * - [3, 2, 1] => [2, 1, 0] + * - [0, 1, 0] => [0, 0, 0] + * + * Provided to enable compatibility for segments written under {@link NullHandling#sqlCompatible()} mode but + * read under {@link NullHandling#replaceWithDefault()} mode. + * + * @see NullHandling#mustCombineNullAndEmptyInDictionary(Indexed) + */ +public class CombineFirstTwoValuesIndexedInts implements IndexedInts +{ + private static final int ZERO_ID = 0; + + IndexedInts delegate; + + public CombineFirstTwoValuesIndexedInts(IndexedInts delegate) + { + this.delegate = delegate; + } + + @Override + public int size() + { + return delegate.size(); + } + + @Override + public int get(int index) + { + final int i = delegate.get(index); + if (i == ZERO_ID) { + return i; + } else { + return i - 1; + } + } + + @Override + public void get(int[] out, int start, int length) + { + delegate.get(out, start, length); + + for (int i = 0; i < length; i++) { + if (out[i] != ZERO_ID) { + out[i]--; + } + } + } + + @Override + public void get(int[] out, int[] indexes, int length) + { + delegate.get(out, indexes, length); + + for (int i = 0; i < length; i++) { + if (out[i] != ZERO_ID) { + out[i]--; + } + } + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + delegate.inspectRuntimeShape(inspector); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java index f48dcfc61b2..a8148ca6a25 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java @@ -20,17 +20,20 @@ package org.apache.druid.segment.serde; import com.google.common.base.Supplier; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.segment.column.DictionaryEncodedColumn; import org.apache.druid.segment.column.StringDictionaryEncodedColumn; import org.apache.druid.segment.data.CachingIndexed; import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.ColumnarMultiInts; import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.Indexed; import javax.annotation.Nullable; import java.nio.ByteBuffer; /** + * */ public class DictionaryEncodedColumnSupplier implements Supplier> { @@ -58,11 +61,40 @@ public class DictionaryEncodedColumnSupplier implements Supplier get() { - return new StringDictionaryEncodedColumn( - singleValuedColumn != null ? singleValuedColumn.get() : null, - multiValuedColumn != null ? multiValuedColumn.get() : null, - new CachingIndexed<>(dictionary, lookupCacheSize), - dictionaryUtf8.singleThreaded() - ); + final Indexed cacheWrappedDictionary; + final Indexed singleThreadedDictionaryUtf8 = dictionaryUtf8.singleThreaded(); + + if (lookupCacheSize > 0) { + cacheWrappedDictionary = new CachingIndexed<>( + dictionary.singleThreaded(), + s -> s == null ? 0 : s.length() * Character.BYTES, + lookupCacheSize + ); + } else { + cacheWrappedDictionary = dictionary.singleThreaded(); + } + + if (NullHandling.mustCombineNullAndEmptyInDictionary(singleThreadedDictionaryUtf8)) { + return new StringDictionaryEncodedColumn( + singleValuedColumn != null ? new CombineFirstTwoValuesColumnarInts(singleValuedColumn.get()) : null, + multiValuedColumn != null ? new CombineFirstTwoValuesColumnarMultiInts(multiValuedColumn.get()) : null, + CombineFirstTwoEntriesIndexed.returnNull(cacheWrappedDictionary), + CombineFirstTwoEntriesIndexed.returnNull(singleThreadedDictionaryUtf8) + ); + } else if (NullHandling.mustReplaceFirstValueWithNullInDictionary(singleThreadedDictionaryUtf8)) { + return new StringDictionaryEncodedColumn( + singleValuedColumn != null ? singleValuedColumn.get() : null, + multiValuedColumn != null ? multiValuedColumn.get() : null, + new ReplaceFirstValueWithNullIndexed<>(cacheWrappedDictionary), + new ReplaceFirstValueWithNullIndexed<>(singleThreadedDictionaryUtf8) + ); + } else { + return new StringDictionaryEncodedColumn( + singleValuedColumn != null ? singleValuedColumn.get() : null, + multiValuedColumn != null ? multiValuedColumn.get() : null, + cacheWrappedDictionary, + singleThreadedDictionaryUtf8 + ); + } } } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplier.java index ae2a1e14823..d7dc25b9796 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplier.java @@ -75,13 +75,24 @@ public class DictionaryEncodedStringIndexSupplier implements ColumnIndexSupplier public T as(Class clazz) { if (bitmaps != null) { - final Indexed singleThreadedStrings = dictionary.singleThreaded(); - final Indexed singleThreadedUtf8 = dictionaryUtf8.singleThreaded(); - final Indexed singleThreadedBitmaps = bitmaps.singleThreaded(); + Indexed singleThreadedStrings = dictionary.singleThreaded(); + Indexed singleThreadedUtf8 = dictionaryUtf8.singleThreaded(); + Indexed singleThreadedBitmaps = bitmaps.singleThreaded(); + + if (NullHandling.mustCombineNullAndEmptyInDictionary(singleThreadedUtf8)) { + singleThreadedStrings = CombineFirstTwoEntriesIndexed.returnNull(singleThreadedStrings); + singleThreadedUtf8 = CombineFirstTwoEntriesIndexed.returnNull(singleThreadedUtf8); + singleThreadedBitmaps = CombineFirstTwoEntriesIndexed.unionBitmaps(bitmapFactory, singleThreadedBitmaps); + } else if (NullHandling.mustReplaceFirstValueWithNullInDictionary(singleThreadedUtf8)) { + singleThreadedStrings = new ReplaceFirstValueWithNullIndexed<>(singleThreadedStrings); + singleThreadedUtf8 = new ReplaceFirstValueWithNullIndexed<>(singleThreadedUtf8); + } + if (clazz.equals(NullValueIndex.class)) { final BitmapColumnIndex nullIndex; - if (NullHandling.isNullOrEquivalent(dictionary.get(0))) { - nullIndex = new SimpleImmutableBitmapIndex(bitmaps.get(0)); + final ByteBuffer firstValue = singleThreadedUtf8.get(0); + if (NullHandling.isNullOrEquivalent(firstValue)) { + nullIndex = new SimpleImmutableBitmapIndex(singleThreadedBitmaps.get(0)); } else { nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap()); } @@ -97,13 +108,14 @@ public class DictionaryEncodedStringIndexSupplier implements ColumnIndexSupplier bitmapFactory, singleThreadedUtf8, singleThreadedBitmaps, - NullHandling.isNullOrEquivalent(dictionary.get(0)) + NullHandling.isNullOrEquivalent(singleThreadedStrings.get(0)) ); - } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) || clazz.equals(DictionaryEncodedValueIndex.class)) { + } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) + || clazz.equals(DictionaryEncodedValueIndex.class)) { return (T) new IndexedStringDictionaryEncodedStringValueIndex<>( bitmapFactory, singleThreadedStrings, - bitmaps + singleThreadedBitmaps ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java index 3e851484b2c..5eda4f84ae9 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Supplier; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerde; @@ -154,7 +155,14 @@ public class DoubleNumericColumnPartSerdeV2 implements ColumnPartSerde final ImmutableBitmap bitmap; final boolean hasNulls; if (buffer.hasRemaining()) { - bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + if (NullHandling.sqlCompatible()) { + bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + } else { + // Read from the buffer (to advance its position) but do not actually retain the bitmaps. + bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); + } + hasNulls = !bitmap.isEmpty(); } else { bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); diff --git a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java index 29a1e2c46fe..8e79bc7a3fa 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java @@ -22,6 +22,7 @@ package org.apache.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerde; @@ -150,8 +151,15 @@ public class FloatNumericColumnPartSerdeV2 implements ColumnPartSerde buffer.position(initialPos + offset); final ImmutableBitmap bitmap; final boolean hasNulls; - if (buffer.hasRemaining()) { - bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + if (buffer.hasRemaining() && NullHandling.sqlCompatible()) { + if (NullHandling.sqlCompatible()) { + bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + } else { + // Read from the buffer (to advance its position) but do not actually retain the bitmaps. + bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); + } + hasNulls = !bitmap.isEmpty(); } else { bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); diff --git a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java index c59bb99b1c7..7f145b36a41 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java @@ -22,6 +22,7 @@ package org.apache.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerde; @@ -152,8 +153,15 @@ public class LongNumericColumnPartSerdeV2 implements ColumnPartSerde buffer.position(initialPos + offset); final ImmutableBitmap bitmap; final boolean hasNulls; - if (buffer.hasRemaining()) { - bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + if (buffer.hasRemaining() && NullHandling.sqlCompatible()) { + if (NullHandling.sqlCompatible()) { + bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + } else { + // Read from the buffer (to advance its position) but do not actually retain the bitmaps. + bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); + } + hasNulls = !bitmap.isEmpty(); } else { bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); diff --git a/processing/src/main/java/org/apache/druid/segment/serde/ReplaceFirstValueWithNullIndexed.java b/processing/src/main/java/org/apache/druid/segment/serde/ReplaceFirstValueWithNullIndexed.java new file mode 100644 index 00000000000..be9e1385d94 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/ReplaceFirstValueWithNullIndexed.java @@ -0,0 +1,131 @@ +/* + * 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.druid.segment.serde; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.data.Indexed; + +import javax.annotation.Nullable; +import java.util.Iterator; + +/** + * An Indexed that replaces the first value with a literal null. + * + * Provided to enable compatibility for segments written under {@link NullHandling#sqlCompatible()} mode but + * read under {@link NullHandling#replaceWithDefault()} mode. + * + * Important note: {@link #isSorted()} returns the same value as the underlying delegate. In this case, this class + * assumes that {@code null} is the lowest possible value in the universe: including anything in {@link #delegate} and + * anything that might be passed to {@link #indexOf(Object)}. Callers must ensure that this precondition is met. + * + * @see NullHandling#mustReplaceFirstValueWithNullInDictionary(Indexed) + */ +public class ReplaceFirstValueWithNullIndexed implements Indexed +{ + private final Indexed delegate; + + public ReplaceFirstValueWithNullIndexed(Indexed delegate) + { + this.delegate = delegate; + + if (delegate.size() < 1) { + throw new ISE("Size[%s] must be >= 1", delegate.size()); + } + } + + @Override + public int size() + { + return delegate.size(); + } + + @Nullable + @Override + public T get(int index) + { + if (index == 0) { + return null; + } else { + return delegate.get(index); + } + } + + @Override + public int indexOf(@Nullable T value) + { + if (value == null) { + return 0; + } else { + final int result = delegate.indexOf(value); + if (result == 0 || result == -1) { + return -2; + } else { + return result; + } + } + } + + @Override + public boolean isSorted() + { + return delegate.isSorted(); + } + + @Override + public Iterator iterator() + { + final Iterator it = delegate.iterator(); + + // Skip first value. + it.next(); + + class ReplaceFirstValueWithNullIndexedIterator implements Iterator + { + boolean returnedNull; + + @Override + public boolean hasNext() + { + return !returnedNull || it.hasNext(); + } + + @Override + public T next() + { + if (!returnedNull) { + returnedNull = true; + return null; + } else { + return it.next(); + } + } + } + + return new ReplaceFirstValueWithNullIndexedIterator(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + delegate.inspectRuntimeShape(inspector); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedColumnIndexSupplier.java index 894ddb55ffc..a617e3b409f 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedColumnIndexSupplier.java @@ -44,11 +44,11 @@ import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; import javax.annotation.Nullable; +import java.nio.ByteBuffer; public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier { private final BitmapFactory bitmapFactory; - private final Supplier dictionary; private final Supplier utf8Dictionary; @Nullable @@ -67,21 +67,30 @@ public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier this.bitmapFactory = bitmapFactory; this.bitmaps = bitmaps; this.utf8Dictionary = utf8Dictionary; - this.dictionary = () -> new StringEncodingStrategies.Utf8ToStringIndexed(this.utf8Dictionary.get()); this.indexedTree = indexedTree; } @Nullable @Override + @SuppressWarnings("unchecked") public T as(Class clazz) { if (bitmaps != null) { - final Indexed singleThreadedBitmaps = bitmaps.singleThreaded(); + Indexed dict = utf8Dictionary.get(); + Indexed singleThreadedBitmaps = bitmaps.singleThreaded(); + + if (NullHandling.mustCombineNullAndEmptyInDictionary(dict)) { + dict = CombineFirstTwoEntriesIndexed.returnNull(dict); + singleThreadedBitmaps = CombineFirstTwoEntriesIndexed.unionBitmaps(bitmapFactory, singleThreadedBitmaps); + } else if (NullHandling.mustReplaceFirstValueWithNullInDictionary(dict)) { + dict = new ReplaceFirstValueWithNullIndexed<>(dict); + } + if (clazz.equals(NullValueIndex.class)) { final BitmapColumnIndex nullIndex; - final StringEncodingStrategies.Utf8ToStringIndexed stringDictionary = dictionary.get(); - if (NullHandling.isNullOrEquivalent(stringDictionary.get(0))) { - nullIndex = new SimpleImmutableBitmapIndex(bitmaps.get(0)); + final ByteBuffer firstValue = dict.get(0); + if (NullHandling.isNullOrEquivalent(firstValue)) { + nullIndex = new SimpleImmutableBitmapIndex(singleThreadedBitmaps.get(0)); } else { nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap()); } @@ -89,17 +98,16 @@ public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier } else if (clazz.equals(StringValueSetIndex.class)) { return (T) new IndexedUtf8ValueSetIndex<>( bitmapFactory, - utf8Dictionary.get(), + dict, singleThreadedBitmaps ); } else if (clazz.equals(DruidPredicateIndex.class)) { return (T) new IndexedStringDruidPredicateIndex<>( bitmapFactory, - dictionary.get(), + new StringEncodingStrategies.Utf8ToStringIndexed(dict), singleThreadedBitmaps ); } else if (clazz.equals(LexicographicalRangeIndex.class)) { - final FrontCodedIndexed dict = utf8Dictionary.get(); return (T) new IndexedUtf8LexicographicalRangeIndex<>( bitmapFactory, dict, @@ -108,10 +116,11 @@ public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier ); } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) || clazz.equals(DictionaryEncodedValueIndex.class)) { + // Need string dictionary instead of UTF8 dictionary return (T) new IndexedStringDictionaryEncodedStringValueIndex<>( bitmapFactory, - dictionary.get(), - bitmaps + new StringEncodingStrategies.Utf8ToStringIndexed(dict), + singleThreadedBitmaps ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java index d67730de436..23bc28acff1 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java @@ -20,9 +20,10 @@ package org.apache.druid.segment.serde; import com.google.common.base.Supplier; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.segment.column.DictionaryEncodedColumn; import org.apache.druid.segment.column.StringDictionaryEncodedColumn; -import org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.ColumnarMultiInts; import org.apache.druid.segment.data.FrontCodedIndexed; @@ -30,7 +31,7 @@ import org.apache.druid.segment.data.FrontCodedIndexed; import javax.annotation.Nullable; /** - * {@link DictionaryEncodedColumnSupplier} but for columns using a {@link StringFrontCodedDictionaryEncodedColumn} + * {@link DictionaryEncodedColumnSupplier} but for columns using a {@link StringUtf8DictionaryEncodedColumn} * instead of the traditional {@link StringDictionaryEncodedColumn} */ public class StringFrontCodedDictionaryEncodedColumnSupplier implements Supplier> @@ -53,10 +54,26 @@ public class StringFrontCodedDictionaryEncodedColumnSupplier implements Supplier @Override public DictionaryEncodedColumn get() { - return new StringFrontCodedDictionaryEncodedColumn( - singleValuedColumn != null ? singleValuedColumn.get() : null, - multiValuedColumn != null ? multiValuedColumn.get() : null, - utf8Dictionary.get() - ); + final FrontCodedIndexed suppliedUtf8Dictionary = utf8Dictionary.get(); + + if (NullHandling.mustCombineNullAndEmptyInDictionary(suppliedUtf8Dictionary)) { + return new StringUtf8DictionaryEncodedColumn( + singleValuedColumn != null ? new CombineFirstTwoValuesColumnarInts(singleValuedColumn.get()) : null, + multiValuedColumn != null ? new CombineFirstTwoValuesColumnarMultiInts(multiValuedColumn.get()) : null, + CombineFirstTwoEntriesIndexed.returnNull(suppliedUtf8Dictionary) + ); + } else if (NullHandling.mustReplaceFirstValueWithNullInDictionary(suppliedUtf8Dictionary)) { + return new StringUtf8DictionaryEncodedColumn( + singleValuedColumn != null ? singleValuedColumn.get() : null, + multiValuedColumn != null ? multiValuedColumn.get() : null, + new ReplaceFirstValueWithNullIndexed<>(suppliedUtf8Dictionary) + ); + } else { + return new StringUtf8DictionaryEncodedColumn( + singleValuedColumn != null ? singleValuedColumn.get() : null, + multiValuedColumn != null ? multiValuedColumn.get() : null, + suppliedUtf8Dictionary + ); + } } } diff --git a/processing/src/test/java/org/apache/druid/common/config/NullHandlingTest.java b/processing/src/test/java/org/apache/druid/common/config/NullHandlingTest.java index 8bfb36d25d3..db9ade70f42 100644 --- a/processing/src/test/java/org/apache/druid/common/config/NullHandlingTest.java +++ b/processing/src/test/java/org/apache/druid/common/config/NullHandlingTest.java @@ -19,10 +19,14 @@ package org.apache.druid.common.config; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; +import java.util.Collections; + import static org.apache.druid.common.config.NullHandling.replaceWithDefault; import static org.junit.Assert.assertEquals; @@ -99,4 +103,86 @@ public final class NullHandlingTest extends InitializedNullHandlingTest NullHandling.initializeForTests(); } } + + @Test + public void test_mustCombineNullAndEmptyInDictionary() + { + Assert.assertFalse( + NullHandling.mustCombineNullAndEmptyInDictionary( + new ListIndexed<>(Collections.singletonList(null)) + ) + ); + + Assert.assertFalse( + NullHandling.mustCombineNullAndEmptyInDictionary( + new ListIndexed<>(StringUtils.toUtf8ByteBuffer("foo")) + ) + ); + + Assert.assertFalse( + NullHandling.mustCombineNullAndEmptyInDictionary( + new ListIndexed<>(StringUtils.toUtf8ByteBuffer("")) + ) + ); + + Assert.assertFalse( + NullHandling.mustCombineNullAndEmptyInDictionary( + new ListIndexed<>(StringUtils.toUtf8ByteBuffer(""), StringUtils.toUtf8ByteBuffer("foo")) + ) + ); + + Assert.assertEquals( + NullHandling.replaceWithDefault(), + NullHandling.mustCombineNullAndEmptyInDictionary( + new ListIndexed<>(null, StringUtils.toUtf8ByteBuffer("")) + ) + ); + + Assert.assertEquals( + NullHandling.replaceWithDefault(), + NullHandling.mustCombineNullAndEmptyInDictionary( + new ListIndexed<>(null, StringUtils.toUtf8ByteBuffer(""), StringUtils.toUtf8ByteBuffer("foo"))) + ); + } + + @Test + public void test_mustReplaceFirstValueWithNullInDictionary() + { + Assert.assertFalse( + NullHandling.mustReplaceFirstValueWithNullInDictionary( + new ListIndexed<>(Collections.singletonList(null)) + ) + ); + + Assert.assertFalse( + NullHandling.mustReplaceFirstValueWithNullInDictionary( + new ListIndexed<>(StringUtils.toUtf8ByteBuffer("foo")) + ) + ); + + Assert.assertEquals( + NullHandling.replaceWithDefault(), + NullHandling.mustReplaceFirstValueWithNullInDictionary( + new ListIndexed<>(StringUtils.toUtf8ByteBuffer("")) + ) + ); + + Assert.assertEquals( + NullHandling.replaceWithDefault(), + NullHandling.mustReplaceFirstValueWithNullInDictionary( + new ListIndexed<>(StringUtils.toUtf8ByteBuffer(""), StringUtils.toUtf8ByteBuffer("foo")) + ) + ); + + Assert.assertFalse( + NullHandling.mustReplaceFirstValueWithNullInDictionary( + new ListIndexed<>(null, StringUtils.toUtf8ByteBuffer("")) + ) + ); + + Assert.assertFalse( + NullHandling.mustReplaceFirstValueWithNullInDictionary( + new ListIndexed<>(null, StringUtils.toUtf8ByteBuffer(""), StringUtils.toUtf8ByteBuffer("foo"))) + ); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index 842e351fa58..8632a6ee01f 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -116,6 +116,11 @@ public class IndexBuilder return new IndexBuilder(jsonMapper, columnConfig); } + public IndexIO getIndexIO() + { + return indexIO; + } + public IndexBuilder schema(IncrementalIndexSchema schema) { this.schema = schema; @@ -198,12 +203,6 @@ public class IndexBuilder return this; } - public IndexBuilder maxRows(int maxRows) - { - this.maxRows = maxRows; - return this; - } - public IndexBuilder intermediaryPersistSize(int rows) { this.intermediatePersistSize = rows; @@ -231,7 +230,7 @@ public class IndexBuilder return buildIncrementalIndexWithRows(schema, maxRows, rows); } - public QueryableIndex buildMMappedIndex() + public File buildMMappedIndexFile() { Preconditions.checkNotNull(indexMerger, "indexMerger"); Preconditions.checkNotNull(tmpDir, "tmpDir"); @@ -255,16 +254,14 @@ public class IndexBuilder // queryable index instead of the incremental index, which also mimics the behavior of real ingestion tasks // which persist incremental indexes as intermediate segments and then merges all the intermediate segments to // publish - return indexIO.loadIndex( - indexMerger.merge( - adapters, - schema.isRollup(), - schema.getMetrics(), - tmpDir, - schema.getDimensionsSpec(), - indexSpec, - Integer.MAX_VALUE - ) + return indexMerger.merge( + adapters, + schema.isRollup(), + schema.getMetrics(), + tmpDir, + schema.getDimensionsSpec(), + indexSpec, + Integer.MAX_VALUE ); } catch (IOException e) { @@ -272,6 +269,15 @@ public class IndexBuilder } } + public QueryableIndex buildMMappedIndex() + { + try { + return indexIO.loadIndex(buildMMappedIndexFile()); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } public QueryableIndex buildMMappedMergedIndex() { diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java index 6568da52c4f..9f0ee07d092 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java @@ -140,7 +140,7 @@ public class IndexMergerNullHandlingTest // Compute all unique values, the same way that IndexMerger is expected to do it. final Set uniqueValues = new HashSet<>(); for (Map m : subsetList) { - final List dValues = normalize(m.get("d"), hasMultipleValues); + final List dValues = normalize(m.get("d")); uniqueValues.addAll(dValues); if (nullFlavors.contains(m)) { @@ -167,7 +167,7 @@ public class IndexMergerNullHandlingTest subsetList.toString(), ImmutableMultiset.copyOf( subsetList.stream() - .map(m -> normalize(m.get("d"), hasMultipleValues)) + .map(m -> normalize(m.get("d"))) .distinct() // Distinct values only, because we expect rollup. .collect(Collectors.toList()) ), @@ -224,7 +224,7 @@ public class IndexMergerNullHandlingTest /** * Normalize an input value the same way that IndexMerger is expected to do it. */ - private static List normalize(final Object value, final boolean hasMultipleValues) + private static List normalize(final Object value) { final List retVal = new ArrayList<>(); @@ -237,7 +237,7 @@ public class IndexMergerNullHandlingTest if (list.isEmpty()) { // empty lists become nulls in single valued columns // they sometimes also become nulls in multi-valued columns (see comments in getRow()) - retVal.add(NullHandling.emptyToNullIfNeeded(null)); + retVal.add(null); } else { retVal.addAll(list.stream().map(NullHandling::emptyToNullIfNeeded).collect(Collectors.toList())); } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index 65c39b359bc..3495d1cb6d9 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -104,6 +104,7 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Rule; import org.junit.rules.TemporaryFolder; @@ -111,6 +112,8 @@ import org.junit.runners.Parameterized; import javax.annotation.Nullable; import java.io.Closeable; +import java.io.File; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -146,11 +149,17 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest static final TimestampSpec DEFAULT_TIMESTAMP_SPEC = new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")); static final DimensionsSpec DEFAULT_DIM_SPEC = new DimensionsSpec( ImmutableList.builder() - .addAll(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3", "timeDim"))) - .add(new DoubleDimensionSchema("d0")) - .add(new FloatDimensionSchema("f0")) - .add(new LongDimensionSchema("l0")) - .build() + .addAll(DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "dim0", + "dim1", + "dim2", + "dim3", + "timeDim" + ))) + .add(new DoubleDimensionSchema("d0")) + .add(new FloatDimensionSchema("f0")) + .add(new LongDimensionSchema("l0")) + .build() ); static final InputRowParser> DEFAULT_PARSER = new MapInputRowParser( @@ -350,32 +359,35 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .put( "mmappedAutoTypesMerged", input -> { - input.mapSchema( - schema -> - new IncrementalIndexSchema( - schema.getMinTimestamp(), - schema.getTimestampSpec(), - schema.getGran(), - schema.getVirtualColumns(), - schema.getDimensionsSpec().withDimensions( - schema.getDimensionsSpec() - .getDimensions() - .stream() - .map( - dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName()) - ) - .collect(Collectors.toList()) - ), - schema.getMetrics(), - schema.isRollup() + final QueryableIndex index = + input + .mapSchema( + schema -> + new IncrementalIndexSchema( + schema.getMinTimestamp(), + schema.getTimestampSpec(), + schema.getGran(), + schema.getVirtualColumns(), + schema.getDimensionsSpec().withDimensions( + schema.getDimensionsSpec() + .getDimensions() + .stream() + .map( + dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName()) + ) + .collect(Collectors.toList()) + ), + schema.getMetrics(), + schema.isRollup() + ) ) - ); - // if 1 row per segment some of the columns have null values for the row which causes 'auto' - // typing default value coercion to be lost in default value mode, so make sure there is at - // least one number in each segment for these tests to pass correctly because the column - // is typeless and so doesn't write out zeros like regular numbers do - input.intermediaryPersistSize(3); - final QueryableIndex index = input.buildMMappedMergedIndex(); + // if 1 row per segment some of the columns have null values for the row which causes 'auto' + // typing default value coercion to be lost in default value mode, so make sure there is at + // least one number in each segment for these tests to pass correctly because the column + // is typeless and so doesn't write out zeros like regular numbers do + .intermediaryPersistSize(3) + .buildMMappedIndex(); + return Pair.of(new QueryableIndexStorageAdapter(index), index); } ) @@ -393,15 +405,38 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest return Pair.of(new QueryableIndexStorageAdapter(index), index); } ) + .put( + "mmappedWithSqlCompatibleNulls", + input -> { + // Build mmapped index in SQL-compatible null handling mode; read it in default-value mode. + Assume.assumeTrue(NullHandling.replaceWithDefault()); + final File file; + try { + NullHandling.initializeForTestsWithValues(false, null); + Assert.assertTrue(NullHandling.sqlCompatible()); + file = input.buildMMappedIndexFile(); + } + finally { + NullHandling.initializeForTests(); + } + + Assert.assertTrue(NullHandling.replaceWithDefault()); + try { + final QueryableIndex index = input.getIndexIO().loadIndex(file); + return Pair.of(new QueryableIndexStorageAdapter(index), index); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ) .put( "rowBasedWithoutTypeSignature", - input -> Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().asStorageAdapter(), () -> { - }) + input -> Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().asStorageAdapter(), () -> {}) ) .put( "rowBasedWithTypeSignature", - input -> Pair.of(input.buildRowBasedSegmentWithTypeSignature().asStorageAdapter(), () -> { - }) + input -> Pair.of(input.buildRowBasedSegmentWithTypeSignature().asStorageAdapter(), () -> {}) ) .put("frame (row-based)", input -> { final FrameSegment segment = input.buildFrameSegment(FrameType.ROW_BASED); @@ -811,7 +846,6 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest } - return values; } } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java index 954e4998535..5b86747aa2a 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -45,6 +45,7 @@ import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.DruidPredicateIndex; import org.apache.druid.segment.column.NullValueIndex; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; @@ -192,7 +193,7 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest bob, NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES ); - try (ScalarStringDictionaryEncodedColumn column = (ScalarStringDictionaryEncodedColumn) supplier.get()) { + try (StringUtf8DictionaryEncodedColumn column = (StringUtf8DictionaryEncodedColumn) supplier.get()) { smokeTest(supplier, column); } } @@ -225,7 +226,7 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest try { threadsStartLatch.await(); for (int iter = 0; iter < 5000; iter++) { - try (ScalarStringDictionaryEncodedColumn column = (ScalarStringDictionaryEncodedColumn) supplier.get()) { + try (StringUtf8DictionaryEncodedColumn column = (StringUtf8DictionaryEncodedColumn) supplier.get()) { smokeTest(supplier, column); } } @@ -241,7 +242,7 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest Assert.assertEquals(expectedReason, failureReason.get()); } - private void smokeTest(ScalarStringColumnAndIndexSupplier supplier, ScalarStringDictionaryEncodedColumn column) + private void smokeTest(ScalarStringColumnAndIndexSupplier supplier, StringUtf8DictionaryEncodedColumn column) { SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size()); ColumnValueSelector valueSelector = column.makeColumnValueSelector(offset); diff --git a/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoEntriesIndexedTest.java b/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoEntriesIndexedTest.java new file mode 100644 index 00000000000..e5dd68978a3 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoEntriesIndexedTest.java @@ -0,0 +1,162 @@ +/* + * 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.druid.segment.serde; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.ListIndexed; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Assert; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import javax.annotation.Nullable; +import java.util.Collections; + +/** + * Test for {@link CombineFirstTwoEntriesIndexed}. + */ +public class CombineFirstTwoEntriesIndexedTest extends InitializedNullHandlingTest +{ + @Test + public void testSizeZero() + { + final IllegalStateException e = Assert.assertThrows( + IllegalStateException.class, + () -> wrap(Indexed.empty(), "xyz") + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Size[0] must be >= 2")) + ); + } + + @Test + public void testSizeOne() + { + final IllegalStateException e = Assert.assertThrows( + IllegalStateException.class, + () -> wrap(new ListIndexed<>("foo"), "xyz") + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Size[1] must be >= 2")) + ); + } + + @Test + public void testSizeTwo() + { + final CombineFirstTwoEntriesIndexed indexed = wrap(new ListIndexed<>("bar", "foo"), "xyz"); + Assert.assertEquals(0, indexed.indexOf("xyz")); + Assert.assertEquals(-2, indexed.indexOf("foo")); + Assert.assertEquals(-2, indexed.indexOf("bar")); + Assert.assertEquals(-2, indexed.indexOf("baz")); + Assert.assertEquals(-2, indexed.indexOf("qux")); + Assert.assertEquals(-2, indexed.indexOf("")); + Assert.assertEquals(-2, indexed.indexOf(null)); + Assert.assertEquals(1, indexed.size()); + Assert.assertEquals("xyz", indexed.get(0)); + Assert.assertFalse(indexed.isSorted()); // Matches delegate. See class-level note in CombineFirstTwoEntriesIndexed. + Assert.assertEquals(ImmutableList.of("xyz"), ImmutableList.copyOf(indexed)); + } + + @Test + public void testSizeThree() + { + final CombineFirstTwoEntriesIndexed indexed = wrap(new ListIndexed<>("bar", "baz", "foo"), "xyz"); + Assert.assertEquals(0, indexed.indexOf("xyz")); + Assert.assertEquals(1, indexed.indexOf("foo")); + Assert.assertEquals(-2, indexed.indexOf("bar")); + Assert.assertEquals(-2, indexed.indexOf("baz")); + Assert.assertEquals(-2, indexed.indexOf("qux")); + Assert.assertEquals(-2, indexed.indexOf("")); + Assert.assertEquals(-2, indexed.indexOf(null)); + Assert.assertEquals("xyz", indexed.get(0)); + Assert.assertEquals("foo", indexed.get(1)); + Assert.assertFalse(indexed.isSorted()); // Matches delegate. See class-level note in CombineFirstTwoEntriesIndexed. + Assert.assertEquals(ImmutableList.of("xyz", "foo"), ImmutableList.copyOf(indexed)); + } + + @Test + public void testSizeTwoSorted() + { + final CombineFirstTwoEntriesIndexed indexed = wrap( + GenericIndexed.fromArray( + new String[]{"bar", "foo"}, + GenericIndexed.STRING_STRATEGY + ), + null + ); + + Assert.assertEquals(0, indexed.indexOf(null)); + Assert.assertEquals(-2, indexed.indexOf("foo")); + Assert.assertEquals(-2, indexed.indexOf("bar")); + Assert.assertEquals(-2, indexed.indexOf("baz")); + Assert.assertEquals(-2, indexed.indexOf("qux")); + Assert.assertEquals(-2, indexed.indexOf("")); + Assert.assertEquals(1, indexed.size()); + Assert.assertNull(indexed.get(0)); + Assert.assertTrue(indexed.isSorted()); // Matches delegate. See class-level note in CombineFirstTwoEntriesIndexed. + Assert.assertEquals(Collections.singletonList(null), Lists.newArrayList(indexed)); + } + + @Test + public void testSizeThreeSorted() + { + final CombineFirstTwoEntriesIndexed indexed = wrap( + GenericIndexed.fromArray( + new String[]{"bar", "baz", "foo"}, + GenericIndexed.STRING_STRATEGY + ), + null + ); + + Assert.assertEquals(0, indexed.indexOf(null)); + Assert.assertEquals(1, indexed.indexOf("foo")); + Assert.assertEquals(-2, indexed.indexOf("bar")); + Assert.assertEquals(-2, indexed.indexOf("baz")); + Assert.assertEquals(-3, indexed.indexOf("qux")); + Assert.assertEquals(-2, indexed.indexOf("")); + Assert.assertEquals(2, indexed.size()); + Assert.assertNull(indexed.get(0)); + Assert.assertEquals("foo", indexed.get(1)); + Assert.assertTrue(indexed.isSorted()); // Matches delegate. See class-level note in CombineFirstTwoEntriesIndexed. + Assert.assertEquals(Lists.newArrayList(null, "foo"), Lists.newArrayList(indexed)); + } + + private CombineFirstTwoEntriesIndexed wrap(final Indexed indexed, @Nullable final T newFirstValue) + { + return new CombineFirstTwoEntriesIndexed(indexed) + { + @Override + protected T newFirstValue() + { + return newFirstValue; + } + }; + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarIntsTest.java b/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarIntsTest.java new file mode 100644 index 00000000000..fee6a839501 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarIntsTest.java @@ -0,0 +1,47 @@ +/* + * 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.druid.segment.serde; + +import org.apache.druid.segment.data.ArrayBasedIndexedInts; +import org.junit.Test; + +/** + * Test for {@link CombineFirstTwoValuesColumnarInts}. + */ +public class CombineFirstTwoValuesColumnarIntsTest +{ + @Test + public void testCombineFirstTwoValues() + { + // (expectedCombined, original) + assertCombine(new int[]{0, 1, 2}, new int[]{1, 2, 3}); + assertCombine(new int[]{0, 0, 1, 2}, new int[]{0, 1, 2, 3}); + assertCombine(new int[]{2, 0, 1, 0, 4, 0}, new int[]{3, 0, 2, 1, 5, 0}); + } + + private static void assertCombine(final int[] expectedCombined, final int[] original) + { + CombineFirstTwoValuesIndexedIntsTest.assertCombine( + expectedCombined, + original, + arr -> new CombineFirstTwoValuesIndexedInts(new ArrayBasedIndexedInts(arr)) + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarMultiIntsTest.java b/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarMultiIntsTest.java new file mode 100644 index 00000000000..de8edf171bf --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesColumnarMultiIntsTest.java @@ -0,0 +1,119 @@ +/* + * 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.druid.segment.serde; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.druid.segment.data.ColumnarMultiInts; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.data.VSizeColumnarInts; +import org.apache.druid.segment.data.VSizeColumnarMultiInts; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; + +/** + * Test for {@link CombineFirstTwoValuesColumnarMultiInts}. + */ +public class CombineFirstTwoValuesColumnarMultiIntsTest +{ + private ColumnarMultiInts original; + private CombineFirstTwoValuesColumnarMultiInts combined; + + @Before + public void setUp() + { + original = VSizeColumnarMultiInts.fromIterable( + ImmutableList.of( + VSizeColumnarInts.fromArray(new int[]{1, 2, 3}), + VSizeColumnarInts.fromArray(new int[]{0, 1, 2, 3}), + VSizeColumnarInts.fromArray(new int[]{3, 0, 2, 1, 5, 0}) + ) + ); + + combined = new CombineFirstTwoValuesColumnarMultiInts(original); + } + + @Test + public void testSize() + { + Assert.assertEquals(original.size(), combined.size()); + } + + @Test + public void testGet() + { + assertEquals(new int[]{0, 1, 2}, combined.get(0)); + assertEquals(new int[]{0, 0, 1, 2}, combined.get(1)); + assertEquals(new int[]{2, 0, 1, 0, 4, 0}, combined.get(2)); + + // "get" reuses a holder + Assert.assertSame(combined.get(1), combined.get(0)); + } + + @Test + public void testGetUnshared() + { + assertEquals(new int[]{0, 1, 2}, combined.getUnshared(0)); + assertEquals(new int[]{0, 0, 1, 2}, combined.getUnshared(1)); + assertEquals(new int[]{2, 0, 1, 0, 4, 0}, combined.getUnshared(2)); + + // Unlike "get", "getUnshared" does not reuse a holder + Assert.assertNotSame(combined.getUnshared(1), combined.getUnshared(0)); + } + + @Test + public void testIndexOf() + { + Assert.assertThrows( + UnsupportedOperationException.class, + () -> combined.indexOf(original.get(0)) + ); + } + + @Test + public void testIsSorted() + { + Assert.assertFalse(combined.isSorted()); + } + + @Test + public void testIterator() + { + final List fromIterator = Lists.newArrayList(combined.iterator()); + Assert.assertEquals(3, fromIterator.size()); + assertEquals(new int[]{0, 1, 2}, fromIterator.get(0)); + assertEquals(new int[]{0, 0, 1, 2}, fromIterator.get(1)); + assertEquals(new int[]{2, 0, 1, 0, 4, 0}, fromIterator.get(2)); + } + + public void assertEquals(final int[] expected, final IndexedInts actual) + { + final int sz = actual.size(); + final int[] actualArray = new int[sz]; + for (int i = 0; i < sz; i++) { + actualArray[i] = actual.get(i); + } + + Assert.assertArrayEquals(expected, actualArray); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesIndexedIntsTest.java b/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesIndexedIntsTest.java new file mode 100644 index 00000000000..26a44473599 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/CombineFirstTwoValuesIndexedIntsTest.java @@ -0,0 +1,101 @@ +/* + * 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.druid.segment.serde; + +import it.unimi.dsi.fastutil.ints.IntArrays; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.data.ArrayBasedIndexedInts; +import org.apache.druid.segment.data.IndexedInts; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.function.Function; + +/** + * Test for {@link CombineFirstTwoValuesIndexedInts}. + */ +public class CombineFirstTwoValuesIndexedIntsTest +{ + @Test + public void testCombineFirstTwoValues() + { + // (expectedCombined, original) + assertCombine(new int[]{0, 1, 2}, new int[]{1, 2, 3}); + assertCombine(new int[]{0, 0, 1, 2}, new int[]{0, 1, 2, 3}); + assertCombine(new int[]{2, 0, 1, 0, 4, 0}, new int[]{3, 0, 2, 1, 5, 0}); + } + + private static void assertCombine(final int[] expectedCombined, final int[] original) + { + assertCombine( + expectedCombined, + original, + arr -> new CombineFirstTwoValuesIndexedInts(new ArrayBasedIndexedInts(arr)) + ); + } + + static void assertCombine( + final int[] expectedCombined, + final int[] original, + final Function combineFn + ) + { + final IndexedInts combined = combineFn.apply(original); + + // Check size. + Assert.assertEquals( + StringUtils.format("%s (size)", Arrays.toString(original)), + expectedCombined.length, + combined.size() + ); + + // Check regular get. + final int[] arr = new int[expectedCombined.length]; + for (int i = 0; i < expectedCombined.length; i++) { + arr[i] = combined.get(i); + } + Assert.assertArrayEquals(StringUtils.format("%s (get)", Arrays.toString(original)), expectedCombined, arr); + + // Check contiguous vector get. + Arrays.fill(arr, Integer.MIN_VALUE); + combined.get(arr, 0, arr.length); + Assert.assertArrayEquals( + StringUtils.format("%s (contiguous vector get)", Arrays.toString(original)), + expectedCombined, + arr + ); + + // Check noncontiguous vector get. + final int[] indexes = new int[expectedCombined.length]; + for (int i = 0; i < expectedCombined.length; i++) { + indexes[indexes.length - 1 - i] = i; // Fetch backwards. + } + + Arrays.fill(arr, Integer.MIN_VALUE); + combined.get(arr, indexes, arr.length); + final int[] expectedCombinedReversed = IntArrays.reverse(IntArrays.copy(expectedCombined)); + Assert.assertArrayEquals( + StringUtils.format("%s (noncontiguous vector get, reversed)", Arrays.toString(original)), + expectedCombinedReversed, + arr + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/ReplaceFirstValueWithNullIndexedTest.java b/processing/src/test/java/org/apache/druid/segment/serde/ReplaceFirstValueWithNullIndexedTest.java new file mode 100644 index 00000000000..cc01ac6589d --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/ReplaceFirstValueWithNullIndexedTest.java @@ -0,0 +1,137 @@ +/* + * 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.druid.segment.serde; + +import com.google.common.collect.Lists; +import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.ListIndexed; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Assert; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import java.util.Collections; + +/** + * Test for {@link ReplaceFirstValueWithNullIndexed}. + */ +public class ReplaceFirstValueWithNullIndexedTest extends InitializedNullHandlingTest +{ + @Test + public void testSizeZero() + { + final IllegalStateException e = Assert.assertThrows( + IllegalStateException.class, + () -> new ReplaceFirstValueWithNullIndexed<>(Indexed.empty()) + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Size[0] must be >= 1")) + ); + } + + @Test + public void testSizeOne() + { + final ReplaceFirstValueWithNullIndexed indexed = + new ReplaceFirstValueWithNullIndexed<>(new ListIndexed<>("bar")); + + Assert.assertEquals(0, indexed.indexOf(null)); + Assert.assertEquals(-2, indexed.indexOf("")); + Assert.assertEquals(-2, indexed.indexOf("foo")); + Assert.assertEquals(-2, indexed.indexOf("bar")); + Assert.assertEquals(-2, indexed.indexOf("baz")); + Assert.assertEquals(-2, indexed.indexOf("qux")); + Assert.assertEquals(1, indexed.size()); + Assert.assertNull(indexed.get(0)); + Assert.assertFalse(indexed.isSorted()); // Matches delegate. See class doc for ReplaceFirstValueWithNullIndexed. + Assert.assertEquals(Collections.singletonList(null), Lists.newArrayList(indexed)); + } + + @Test + public void testSizeTwo() + { + final ReplaceFirstValueWithNullIndexed indexed = + new ReplaceFirstValueWithNullIndexed<>(new ListIndexed<>("bar", "foo")); + + Assert.assertEquals(0, indexed.indexOf(null)); + Assert.assertEquals(1, indexed.indexOf("foo")); + Assert.assertEquals(-2, indexed.indexOf("")); + Assert.assertEquals(-2, indexed.indexOf("bar")); + Assert.assertEquals(-2, indexed.indexOf("baz")); + Assert.assertEquals(-2, indexed.indexOf("qux")); + Assert.assertEquals(2, indexed.size()); + Assert.assertNull(indexed.get(0)); + Assert.assertEquals("foo", indexed.get(1)); + Assert.assertFalse(indexed.isSorted()); // Matches delegate. See class doc for ReplaceFirstValueWithNullIndexed. + Assert.assertEquals(Lists.newArrayList(null, "foo"), Lists.newArrayList(indexed)); + } + + @Test + public void testSizeOneSorted() + { + final ReplaceFirstValueWithNullIndexed indexed = + new ReplaceFirstValueWithNullIndexed<>( + GenericIndexed.fromArray( + new String[]{"bar"}, + GenericIndexed.STRING_STRATEGY + ) + ); + + Assert.assertEquals(0, indexed.indexOf(null)); + Assert.assertEquals(-2, indexed.indexOf("")); + Assert.assertEquals(-2, indexed.indexOf("foo")); + Assert.assertEquals(-2, indexed.indexOf("bar")); + Assert.assertEquals(-2, indexed.indexOf("baz")); + Assert.assertEquals(-2, indexed.indexOf("qux")); + Assert.assertEquals(1, indexed.size()); + Assert.assertNull(indexed.get(0)); + Assert.assertTrue(indexed.isSorted()); // Matches delegate. See class doc for ReplaceFirstValueWithNullIndexed. + Assert.assertEquals(Collections.singletonList(null), Lists.newArrayList(indexed)); + } + + @Test + public void testSizeTwoSorted() + { + final ReplaceFirstValueWithNullIndexed indexed = + new ReplaceFirstValueWithNullIndexed<>( + GenericIndexed.fromArray( + new String[]{"bar", "foo"}, + GenericIndexed.STRING_STRATEGY + ) + ); + + Assert.assertEquals(0, indexed.indexOf(null)); + Assert.assertEquals(1, indexed.indexOf("foo")); + Assert.assertEquals(-2, indexed.indexOf("")); + Assert.assertEquals(-2, indexed.indexOf("bar")); + Assert.assertEquals(-2, indexed.indexOf("baz")); + Assert.assertEquals(-3, indexed.indexOf("qux")); + Assert.assertEquals(2, indexed.size()); + Assert.assertNull(indexed.get(0)); + Assert.assertEquals("foo", indexed.get(1)); + Assert.assertTrue(indexed.isSorted()); // Matches delegate. See class doc for ReplaceFirstValueWithNullIndexed. + Assert.assertEquals(Lists.newArrayList(null, "foo"), Lists.newArrayList(indexed)); + } +} From e552f68e774cf8727e6bf504fed9c94fa3a0a849 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 28 Jun 2023 10:56:38 -0700 Subject: [PATCH 46/74] Update .asf.yaml to simplify email subjects. (#14494) Currently, github notification emails to commits@druid.apache.org have a wide variety of subject lines. This foils email client threading and makes the firehose of emails hard to follow. This is an attempt, inspired by the config for plc4x, to make the emails easier to follow. After this change, it may even make sense to send some to dev@ instead of commits@. --- .asf.yaml | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/.asf.yaml b/.asf.yaml index a041df443df..d80bca07e48 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -1,3 +1,34 @@ +# note: documentation can be found here https://cwiki.apache.org/confluence/display/INFRA/Git+-+.asf.yaml+features + github: description: 'Apache Druid: a high performance real-time analytics database.' homepage: https://druid.apache.org/ + + # Attempt to make the auto-generated github emails more easily readable in email clients. + # Configuration adapted from https://github.com/apache/plc4x/blob/develop/.asf.yaml. + custom_subjects: + new_pr: "[PR] {title}" + close_pr: "Re: [PR] {title}" + comment_pr: "Re: [PR] {title}" + diffcomment: "Re: [PR] {title}" + merge_pr: "Re: [PR] {title}" + new_issue: "[I] {title}" + comment_issue: "Re: [I] {title}" + close_issue: "Re: [I] {title}" + catchall: "[GH] {title}" + new_discussion: "[D] {title}" + edit_discussion: "Re: [D] {title}" + close_discussion: "Re: [D] {title}" + close_discussion_with_comment: "Re: [D] {title}" + reopen_discussion: "Re: [D] {title}" + new_comment_discussion: "Re: [D] {title}" + edit_comment_discussion: "Re: [D] {title}" + delete_comment_discussion: "Re: [D] {title}" + +notifications: + commits: commits@druid.apache.org + issues: commits@druid.apache.org + discussions: commits@druid.apache.org + pullrequests_status: commits@druid.apache.org + pullrequests_comment: commits@druid.apache.org + pullrequests_bot_dependabot: commits@druid.apache.org From c36f12f1d8bf81cf110dd41853627be175816b00 Mon Sep 17 00:00:00 2001 From: Jonathan Wei Date: Wed, 28 Jun 2023 13:14:19 -0500 Subject: [PATCH 47/74] Support complex variance object inputs for variance SQL agg function (#14463) * Support complex variance object inputs for variance SQL agg function * Add test * Include complexTypeChecker, address PR comments * Checkstyle, javadoc link --- .../variance/VarianceAggregatorFactory.java | 2 +- .../sql/BaseVarianceSqlAggregator.java | 67 ++++++++++++--- .../sql/VarianceSqlAggregatorTest.java | 58 ++++++++++++- .../sql/calcite/table/RowSignatures.java | 82 +++++++++++++++++-- 4 files changed, 188 insertions(+), 21 deletions(-) diff --git a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java index 47eccfbffd5..40d06bbbe02 100644 --- a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java +++ b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java @@ -60,7 +60,7 @@ import java.util.Objects; @JsonTypeName("variance") public class VarianceAggregatorFactory extends AggregatorFactory { - private static final String VARIANCE_TYPE_NAME = "variance"; + public static final String VARIANCE_TYPE_NAME = "variance"; public static final ColumnType TYPE = ColumnType.ofComplex(VARIANCE_TYPE_NAME); protected final String fieldName; diff --git a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java index 3eb3f498161..0b1562eb83d 100644 --- a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java +++ b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java @@ -26,7 +26,10 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlAggFunction; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.type.OperandTypes; +import org.apache.calcite.sql.type.ReturnTypes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -42,15 +45,33 @@ import org.apache.druid.sql.calcite.aggregation.Aggregations; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; +import org.apache.druid.sql.calcite.expression.OperatorConversions; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; +import org.apache.druid.sql.calcite.table.RowSignatures; import javax.annotation.Nullable; import java.util.List; public abstract class BaseVarianceSqlAggregator implements SqlAggregator { + private static final String VARIANCE_NAME = "VARIANCE"; + private static final String STDDEV_NAME = "STDDEV"; + + private static final SqlAggFunction VARIANCE_SQL_AGG_FUNC_INSTANCE = + buildSqlAvgAggFunction(VARIANCE_NAME); + private static final SqlAggFunction VARIANCE_POP_SQL_AGG_FUNC_INSTANCE = + buildSqlAvgAggFunction(SqlKind.VAR_POP.name()); + private static final SqlAggFunction VARIANCE_SAMP_SQL_AGG_FUNC_INSTANCE = + buildSqlAvgAggFunction(SqlKind.VAR_SAMP.name()); + private static final SqlAggFunction STDDEV_SQL_AGG_FUNC_INSTANCE = + buildSqlAvgAggFunction(STDDEV_NAME); + private static final SqlAggFunction STDDEV_POP_SQL_AGG_FUNC_INSTANCE = + buildSqlAvgAggFunction(SqlKind.STDDEV_POP.name()); + private static final SqlAggFunction STDDEV_SAMP_SQL_AGG_FUNC_INSTANCE = + buildSqlAvgAggFunction(SqlKind.STDDEV_SAMP.name()); + @Nullable @Override public Aggregation toDruidAggregation( @@ -104,12 +125,13 @@ public abstract class BaseVarianceSqlAggregator implements SqlAggregator if (inputType.isNumeric()) { inputTypeName = StringUtils.toLowerCase(inputType.getType().name()); + } else if (inputType.equals(VarianceAggregatorFactory.TYPE)) { + inputTypeName = VarianceAggregatorFactory.VARIANCE_TYPE_NAME; } else { throw new IAE("VarianceSqlAggregator[%s] has invalid inputType[%s]", func, inputType.asTypeString()); } - - if (func == SqlStdOperatorTable.VAR_POP || func == SqlStdOperatorTable.STDDEV_POP) { + if (func.getName().equals(SqlKind.VAR_POP.name()) || func.getName().equals(SqlKind.STDDEV_POP.name())) { estimator = "population"; } else { estimator = "sample"; @@ -122,9 +144,9 @@ public abstract class BaseVarianceSqlAggregator implements SqlAggregator inputTypeName ); - if (func == SqlStdOperatorTable.STDDEV_POP - || func == SqlStdOperatorTable.STDDEV_SAMP - || func == SqlStdOperatorTable.STDDEV) { + if (func.getName().equals(STDDEV_NAME) + || func.getName().equals(SqlKind.STDDEV_POP.name()) + || func.getName().equals(SqlKind.STDDEV_SAMP.name())) { postAggregator = new StandardDeviationPostAggregator( name, aggregatorFactory.getName(), @@ -137,21 +159,40 @@ public abstract class BaseVarianceSqlAggregator implements SqlAggregator ); } + /** + * Creates a {@link SqlAggFunction} that is the same as {@link org.apache.calcite.sql.fun.SqlAvgAggFunction} + * but with an operand type that accepts variance aggregator objects in addition to numeric inputs. + */ + private static SqlAggFunction buildSqlAvgAggFunction(String name) + { + return OperatorConversions + .aggregatorBuilder(name) + .returnTypeInference(ReturnTypes.AVG_AGG_FUNCTION) + .operandTypeChecker( + OperandTypes.or( + OperandTypes.NUMERIC, + RowSignatures.complexTypeChecker(VarianceAggregatorFactory.TYPE) + ) + ) + .functionCategory(SqlFunctionCategory.NUMERIC) + .build(); + } + public static class VarPopSqlAggregator extends BaseVarianceSqlAggregator { @Override public SqlAggFunction calciteFunction() { - return SqlStdOperatorTable.VAR_POP; + return VARIANCE_POP_SQL_AGG_FUNC_INSTANCE; } } - + public static class VarSampSqlAggregator extends BaseVarianceSqlAggregator { @Override public SqlAggFunction calciteFunction() { - return SqlStdOperatorTable.VAR_SAMP; + return VARIANCE_SAMP_SQL_AGG_FUNC_INSTANCE; } } @@ -160,7 +201,7 @@ public abstract class BaseVarianceSqlAggregator implements SqlAggregator @Override public SqlAggFunction calciteFunction() { - return SqlStdOperatorTable.VARIANCE; + return VARIANCE_SQL_AGG_FUNC_INSTANCE; } } @@ -169,7 +210,7 @@ public abstract class BaseVarianceSqlAggregator implements SqlAggregator @Override public SqlAggFunction calciteFunction() { - return SqlStdOperatorTable.STDDEV_POP; + return STDDEV_POP_SQL_AGG_FUNC_INSTANCE; } } @@ -178,7 +219,7 @@ public abstract class BaseVarianceSqlAggregator implements SqlAggregator @Override public SqlAggFunction calciteFunction() { - return SqlStdOperatorTable.STDDEV_SAMP; + return STDDEV_SAMP_SQL_AGG_FUNC_INSTANCE; } } @@ -187,7 +228,7 @@ public abstract class BaseVarianceSqlAggregator implements SqlAggregator @Override public SqlAggFunction calciteFunction() { - return SqlStdOperatorTable.STDDEV; + return STDDEV_SQL_AGG_FUNC_INSTANCE; } } } diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java index bc1ef681693..5c496c46635 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java @@ -40,6 +40,7 @@ import org.apache.druid.query.aggregation.stats.DruidStatsModule; import org.apache.druid.query.aggregation.variance.StandardDeviationPostAggregator; import org.apache.druid.query.aggregation.variance.VarianceAggregatorCollector; import org.apache.druid.query.aggregation.variance.VarianceAggregatorFactory; +import org.apache.druid.query.aggregation.variance.VarianceSerde; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; @@ -51,6 +52,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; @@ -82,8 +84,10 @@ public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest final Injector injector ) throws IOException { + ComplexMetrics.registerSerde(VarianceSerde.TYPE_NAME, new VarianceSerde()); + final QueryableIndex index = - IndexBuilder.create() + IndexBuilder.create(CalciteTests.getJsonMapper().registerModules(new DruidStatsModule().getJacksonModules())) .tmpDir(temporaryFolder.newFolder()) .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .schema( @@ -100,7 +104,8 @@ public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest ) .withMetrics( new CountAggregatorFactory("cnt"), - new DoubleSumAggregatorFactory("m1", "m1") + new DoubleSumAggregatorFactory("m1", "m1"), + new VarianceAggregatorFactory("var1", "m1", null, null) ) .withRollup(false) .build() @@ -624,6 +629,55 @@ public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest ); } + @Test + public void testVarianceAggAsInput() + { + final List expectedResults = ImmutableList.of( + new Object[]{ + "3.5", + "2.9166666666666665", + "3.5", + "1.8708286933869707", + "1.707825127659933", + "1.8708286933869707" + } + ); + testQuery( + "SELECT\n" + + "VARIANCE(var1),\n" + + "VAR_POP(var1),\n" + + "VAR_SAMP(var1),\n" + + "STDDEV(var1),\n" + + "STDDEV_POP(var1),\n" + + "STDDEV_SAMP(var1)\n" + + "FROM numfoo", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE3) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .aggregators( + ImmutableList.of( + new VarianceAggregatorFactory("a0:agg", "var1", "sample", "variance"), + new VarianceAggregatorFactory("a1:agg", "var1", "population", "variance"), + new VarianceAggregatorFactory("a2:agg", "var1", "sample", "variance"), + new VarianceAggregatorFactory("a3:agg", "var1", "sample", "variance"), + new VarianceAggregatorFactory("a4:agg", "var1", "population", "variance"), + new VarianceAggregatorFactory("a5:agg", "var1", "sample", "variance") + ) + ) + .postAggregators( + new StandardDeviationPostAggregator("a3", "a3:agg", "sample"), + new StandardDeviationPostAggregator("a4", "a4:agg", "population"), + new StandardDeviationPostAggregator("a5", "a5:agg", "sample") + ) + .context(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT) + .build() + ), + expectedResults + ); + } + @Override public void assertResultsEquals(String sql, List expectedResults, List results) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java b/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java index 32abe56ee8d..87519c75374 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java @@ -23,11 +23,18 @@ import com.google.common.base.Preconditions; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeComparability; import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.sql.SqlCallBinding; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperandCountRange; +import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.type.AbstractSqlType; +import org.apache.calcite.sql.type.SqlOperandCountRanges; +import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.ordering.StringComparator; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.column.ColumnHolder; @@ -79,7 +86,9 @@ public class RowSignatures { Preconditions.checkNotNull(simpleExtraction, "simpleExtraction"); if (simpleExtraction.getExtractionFn() != null - || rowSignature.getColumnType(simpleExtraction.getColumn()).map(type -> type.is(ValueType.STRING)).orElse(false)) { + || rowSignature.getColumnType(simpleExtraction.getColumn()) + .map(type -> type.is(ValueType.STRING)) + .orElse(false)) { return StringComparators.LEXICOGRAPHIC; } else { return StringComparators.NUMERIC; @@ -164,7 +173,7 @@ public class RowSignatures * Creates a {@link ComplexSqlType} using the supplied {@link RelDataTypeFactory} to ensure that the * {@link ComplexSqlType} is interned. This is important because Calcite checks that the references are equal * instead of the objects being equivalent. - * + *

    * This method uses {@link RelDataTypeFactory#createTypeWithNullability(RelDataType, boolean) ensures that if the * type factory is a {@link org.apache.calcite.rel.type.RelDataTypeFactoryImpl} that the type is passed through * {@link org.apache.calcite.rel.type.RelDataTypeFactoryImpl#canonize(RelDataType)} which interns the type. @@ -179,15 +188,15 @@ public class RowSignatures /** * Calcite {@link RelDataType} for Druid complex columns, to preserve complex type information. - * + *

    * If using with other operations of a {@link RelDataTypeFactory}, consider wrapping the creation of this type in * {@link RelDataTypeFactory#createTypeWithNullability(RelDataType, boolean) to ensure that if the type factory is a * {@link org.apache.calcite.rel.type.RelDataTypeFactoryImpl} that the type is passed through * {@link org.apache.calcite.rel.type.RelDataTypeFactoryImpl#canonize(RelDataType)} which interns the type. - * + *

    * If {@link SqlTypeName} is going to be {@link SqlTypeName#OTHER} and a {@link RelDataTypeFactory} is available, * consider using {@link #makeComplexType(RelDataTypeFactory, ColumnType, boolean)}. - * + *

    * This type does not work well with {@link org.apache.calcite.sql.type.ReturnTypes#explicit(RelDataType)}, which * will create new {@link RelDataType} using {@link SqlTypeName} during return type inference, so implementors of * {@link org.apache.druid.sql.calcite.expression.SqlOperatorConversion} should implement the @@ -235,4 +244,67 @@ public class RowSignatures return columnType.asTypeString(); } } + + public static ComplexSqlSingleOperandTypeChecker complexTypeChecker(ColumnType complexType) + { + return new ComplexSqlSingleOperandTypeChecker( + new ComplexSqlType(SqlTypeName.OTHER, complexType, true) + ); + } + + public static final class ComplexSqlSingleOperandTypeChecker implements SqlSingleOperandTypeChecker + { + private final ComplexSqlType type; + + public ComplexSqlSingleOperandTypeChecker( + ComplexSqlType type + ) + { + this.type = type; + } + + @Override + public boolean checkSingleOperandType( + SqlCallBinding callBinding, + SqlNode operand, + int iFormalOperand, + boolean throwOnFailure + ) + { + return type.equals(callBinding.getValidator().deriveType(callBinding.getScope(), operand)); + } + + @Override + public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) + { + if (callBinding.getOperandCount() != 1) { + return false; + } + return checkSingleOperandType(callBinding, callBinding.operand(0), 0, throwOnFailure); + } + + @Override + public SqlOperandCountRange getOperandCountRange() + { + return SqlOperandCountRanges.of(1); + } + + @Override + public String getAllowedSignatures(SqlOperator op, String opName) + { + return StringUtils.format("'%s'(%s)", opName, type); + } + + @Override + public Consistency getConsistency() + { + return Consistency.NONE; + } + + @Override + public boolean isOptional(int i) + { + return false; + } + } } From 34c55a0bde701e18fbe064bfe574caedb752ff4f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 28 Jun 2023 13:43:05 -0700 Subject: [PATCH 48/74] SQL: SUBSTRING support for non-literals. (#14480) * SQL: SUBSTRING support for non-literals. * Fix AssertionError test. * Fix header. --- .../builtin/SubstringOperatorConversion.java | 56 ++++++++--- .../calcite/expression/ExpressionsTest.java | 92 +++++++++++++++++++ .../util/CalciteTestInjectorBuilder.java | 4 +- .../druid/sql/calcite/util/CalciteTests.java | 1 - .../sql/calcite/util/QueryFrameworkUtils.java | 2 +- .../AssertionErrorOperatorConversion.java | 66 +++++++++++++ .../CalciteTestOperatorModule.java | 36 ++++++++ .../druid/sql/http/SqlResourceTest.java | 16 +--- 8 files changed, 246 insertions(+), 27 deletions(-) create mode 100644 sql/src/test/java/org/apache/druid/sql/calcite/util/testoperator/AssertionErrorOperatorConversion.java create mode 100644 sql/src/test/java/org/apache/druid/sql/calcite/util/testoperator/CalciteTestOperatorModule.java diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SubstringOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SubstringOperatorConversion.java index ae470922c1c..41f97b9322d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SubstringOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/SubstringOperatorConversion.java @@ -22,6 +22,7 @@ package org.apache.druid.sql.calcite.expression.builtin; import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexUtil; import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlOperator; @@ -36,6 +37,8 @@ import org.apache.druid.sql.calcite.expression.OperatorConversions; import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; import org.apache.druid.sql.calcite.planner.PlannerContext; +import java.util.List; + public class SubstringOperatorConversion implements SqlOperatorConversion { private static final SqlFunction SQL_FUNCTION = OperatorConversions @@ -63,29 +66,56 @@ public class SubstringOperatorConversion implements SqlOperatorConversion // SQL is 1-indexed, Druid is 0-indexed. final RexCall call = (RexCall) rexNode; - final DruidExpression input = Expressions.toDruidExpression( - plannerContext, - rowSignature, - call.getOperands().get(0) - ); + final List operands = call.getOperands(); + final RexNode inputNode = operands.get(0); + final DruidExpression input = Expressions.toDruidExpression(plannerContext, rowSignature, inputNode); if (input == null) { return null; } - final int index = RexLiteral.intValue(call.getOperands().get(1)) - 1; - final int length; - if (call.getOperands().size() > 2) { - length = RexLiteral.intValue(call.getOperands().get(2)); + + final RexNode indexNode = operands.get(1); + final Integer adjustedIndexLiteral = RexUtil.isLiteral(indexNode, true) ? RexLiteral.intValue(indexNode) - 1 : null; + final String adjustedIndexExpr; + if (adjustedIndexLiteral != null) { + adjustedIndexExpr = String.valueOf(adjustedIndexLiteral); } else { - length = -1; + final DruidExpression indexExpr = Expressions.toDruidExpression(plannerContext, rowSignature, indexNode); + if (indexExpr == null) { + return null; + } + adjustedIndexExpr = StringUtils.format("(%s - 1)", indexExpr.getExpression()); + } + if (adjustedIndexExpr == null) { + return null; + } + + final RexNode lengthNode = operands.size() > 2 ? operands.get(2) : null; + final Integer lengthLiteral = + lengthNode != null && RexUtil.isLiteral(lengthNode, true) ? RexLiteral.intValue(lengthNode) : null; + final String lengthExpr; + if (lengthNode != null) { + final DruidExpression lengthExpression = Expressions.toDruidExpression(plannerContext, rowSignature, lengthNode); + if (lengthExpression == null) { + return null; + } + lengthExpr = lengthExpression.getExpression(); + } else { + lengthExpr = "-1"; } return input.map( - simpleExtraction -> simpleExtraction.cascade(new SubstringDimExtractionFn(index, length < 0 ? null : length)), + simpleExtraction -> { + if (adjustedIndexLiteral != null && (lengthNode == null || lengthLiteral != null)) { + return simpleExtraction.cascade(new SubstringDimExtractionFn(adjustedIndexLiteral, lengthLiteral)); + } else { + return null; + } + }, expression -> StringUtils.format( "substring(%s, %s, %s)", expression, - DruidExpression.longLiteral(index), - DruidExpression.longLiteral(length) + adjustedIndexExpr, + lengthExpr ) ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java index 9b269ecde3c..3fbd517e1cb 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java @@ -34,6 +34,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.math.expr.ExpressionValidationException; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.RegexDimExtractionFn; +import org.apache.druid.query.extraction.SubstringDimExtractionFn; import org.apache.druid.query.filter.RegexDimFilter; import org.apache.druid.query.filter.SearchQueryDimFilter; import org.apache.druid.query.search.ContainsSearchQuerySpec; @@ -55,6 +56,7 @@ import org.apache.druid.sql.calcite.expression.builtin.RightOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RoundOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.StringFormatOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.StrposOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.SubstringOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.TimeCeilOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.TimeExtractOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.TimeFloorOperatorConversion; @@ -167,6 +169,96 @@ public class ExpressionsTest extends ExpressionTestBase ); } + @Test + public void testSubstring() + { + testHelper.testExpressionString( + new SubstringOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral(1), + testHelper.makeLiteral(2) + ), + makeExpression( + SimpleExtraction.of("s", new SubstringDimExtractionFn(0, 2)), + "substring(\"s\", 0, 2)" + ), + "fo" + ); + + testHelper.testExpressionString( + new SubstringOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral(2), + testHelper.makeLiteral(1) + ), + makeExpression( + SimpleExtraction.of("s", new SubstringDimExtractionFn(1, 1)), + "substring(\"s\", 1, 1)" + ), + "o" + ); + + testHelper.testExpressionString( + new SubstringOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral(1) + ), + makeExpression( + SimpleExtraction.of("s", new SubstringDimExtractionFn(0, null)), + "substring(\"s\", 0, -1)" + ), + "foo" + ); + + testHelper.testExpressionString( + new SubstringOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral(2) + ), + makeExpression( + SimpleExtraction.of("s", new SubstringDimExtractionFn(1, null)), + "substring(\"s\", 1, -1)" + ), + "oo" + ); + + testHelper.testExpressionString( + new SubstringOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral(1), + testHelper.makeInputRef("p") // p is 3 + ), + makeExpression("substring(\"s\", 0, \"p\")"), + "foo" + ); + + testHelper.testExpressionString( + new SubstringOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("spacey"), + testHelper.makeInputRef("p") // p is 3 + ), + makeExpression("substring(\"spacey\", (\"p\" - 1), -1)"), + "hey there " + ); + + testHelper.testExpressionString( + new SubstringOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("spacey"), + testHelper.makeInputRef("p"), // p is 3 + testHelper.makeInputRef("p") // p is 3 + ), + makeExpression("substring(\"spacey\", (\"p\" - 1), \"p\")"), + "hey" + ); + } + @Test public void testRegexpExtract() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestInjectorBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestInjectorBuilder.java index 4dc825cac6a..345b8b3ad9b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestInjectorBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestInjectorBuilder.java @@ -26,6 +26,7 @@ import org.apache.druid.initialization.CoreInjectorBuilder; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.sql.calcite.aggregation.SqlAggregationModule; +import org.apache.druid.sql.calcite.util.testoperator.CalciteTestOperatorModule; /** * Create the injector used for {@link CalciteTests#INJECTOR}, but in a way @@ -41,7 +42,8 @@ public class CalciteTestInjectorBuilder extends CoreInjectorBuilder add( new SegmentWranglerModule(), new LookylooModule(), - new SqlAggregationModule() + new SqlAggregationModule(), + new CalciteTestOperatorModule() ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 0dfec3759e0..905c28c5fad 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -327,7 +327,6 @@ public class CalciteTests public static SystemSchema createMockSystemSchema( final DruidSchema druidSchema, final SpecificSegmentsQuerySegmentWalker walker, - final PlannerConfig plannerConfig, final AuthorizerMapper authorizerMapper ) { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java index ed4459ba38b..63235a76c7e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java @@ -146,7 +146,7 @@ public class QueryFrameworkUtils druidSchemaManager ); SystemSchema systemSchema = - CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig, authorizerMapper); + CalciteTests.createMockSystemSchema(druidSchema, walker, authorizerMapper); LookupSchema lookupSchema = createMockLookupSchema(injector); ViewSchema viewSchema = viewManager != null ? new ViewSchema(viewManager) : null; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/testoperator/AssertionErrorOperatorConversion.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/testoperator/AssertionErrorOperatorConversion.java new file mode 100644 index 00000000000..6d4aa44e5e7 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/testoperator/AssertionErrorOperatorConversion.java @@ -0,0 +1,66 @@ +/* + * 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.druid.sql.calcite.util.testoperator; + +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.expression.DruidExpression; +import org.apache.druid.sql.calcite.expression.OperatorConversions; +import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.http.SqlResourceTest; + +import javax.annotation.Nullable; + +/** + * There are various points where Calcite feels it is acceptable to throw an AssertionError when it receives bad + * input. This is unfortunate as a java.lang.Error is very clearly documented to be something that nobody should + * try to catch. But, we can editorialize all we want, we still have to deal with it. So, this operator triggers + * the AssertionError behavior by using RexLiteral.intValue with bad input (a RexNode that is not a literal). + * + * The test {@link SqlResourceTest#testAssertionErrorThrowsErrorWithFilterResponse()} verifies that our exception + * handling deals with this meaningfully. + */ +public class AssertionErrorOperatorConversion implements SqlOperatorConversion +{ + private static final SqlOperator OPERATOR = + OperatorConversions.operatorBuilder("assertion_error") + .operandTypes() + .returnTypeNonNull(SqlTypeName.BIGINT) + .build(); + + @Override + public SqlOperator calciteOperator() + { + return OPERATOR; + } + + @Nullable + @Override + public DruidExpression toDruidExpression(PlannerContext plannerContext, RowSignature rowSignature, RexNode rexNode) + { + // Throws AssertionError. See class-level javadoc for rationale about why we're doing this. + RexLiteral.intValue(rexNode); + return null; + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/testoperator/CalciteTestOperatorModule.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/testoperator/CalciteTestOperatorModule.java new file mode 100644 index 00000000000..16dfac5e3d4 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/testoperator/CalciteTestOperatorModule.java @@ -0,0 +1,36 @@ +/* + * 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.druid.sql.calcite.util.testoperator; + +import com.google.inject.Binder; +import com.google.inject.Module; +import org.apache.druid.sql.guice.SqlBindings; + +/** + * Adds operators that are only used in tests -- not production. + */ +public class CalciteTestOperatorModule implements Module +{ + @Override + public void configure(Binder binder) + { + SqlBindings.addOperatorConversion(binder, AssertionErrorOperatorConversion.class); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index d92b1cbcd44..2c7e002a5f3 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -1589,28 +1589,22 @@ public class SqlResourceTest extends CalciteTestBase } /** - * There are various points where Calcite feels it is acceptable to throw an AssertionError when it receives bad - * input. This is unfortunate as a java.lang.Error is very clearly documented to be something that nobody should - * try to catch. But, we can editorialize all we want, we still have to deal with it. So, this test reproduces - * the AssertionError behavior by using the substr() command. At the time that this test was written, the - * SQL substr assumes a literal for the second argument. The code ends up calling `RexLiteral.intValue` on the - * argument, which ends up calling a method that fails with an AssertionError, so this should generate the - * bad behavior for us. This test is validating that our exception handling deals with this meaningfully. + * See class-level javadoc for {@link org.apache.druid.sql.calcite.util.testoperator.AssertionErrorOperatorConversion} + * for rationale as to why this test exists. + * * If this test starts failing, it could be indicative of us not handling the AssertionErrors well anymore, * OR it could be indicative of this specific code path not throwing an AssertionError anymore. If we run * into the latter case, we should seek out a new code path that generates the error from Calcite. In the best * world, this test starts failing because Calcite has moved all of its execptions away from AssertionErrors * and we can no longer reproduce the behavior through Calcite, in that world, we should remove our own handling * and this test at the same time. - * - * @throws Exception */ @Test public void testAssertionErrorThrowsErrorWithFilterResponse() throws Exception { ErrorResponse exception = postSyncForException( new SqlQuery( - "SELECT *, substr(dim2, strpos(dim2, 'hi')+2, 2) FROM foo LIMIT 2", + "SELECT assertion_error() FROM foo LIMIT 2", ResultFormat.OBJECT, false, false, @@ -1625,7 +1619,7 @@ public class SqlResourceTest extends CalciteTestBase exception.getUnderlyingException(), DruidExceptionMatcher .invalidSqlInput() - .expectMessageIs("Calcite assertion violated: [not a literal: +(STRPOS($2, 'hi'), 2)]") + .expectMessageIs("Calcite assertion violated: [not a literal: assertion_error()]") ); Assert.assertTrue(lifecycleManager.getAll("id").isEmpty()); } From fd1a88a6b3f746f88946fce8a71662b63390ce94 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 28 Jun 2023 15:05:07 -0700 Subject: [PATCH 49/74] .asf.yaml: Add required "repository" field. (#14499) Our new .asf.yaml is not getting picked up because custom_subjects must include "repository". --- .asf.yaml | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index d80bca07e48..9c0a326a08c 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -7,23 +7,23 @@ github: # Attempt to make the auto-generated github emails more easily readable in email clients. # Configuration adapted from https://github.com/apache/plc4x/blob/develop/.asf.yaml. custom_subjects: - new_pr: "[PR] {title}" - close_pr: "Re: [PR] {title}" - comment_pr: "Re: [PR] {title}" - diffcomment: "Re: [PR] {title}" - merge_pr: "Re: [PR] {title}" - new_issue: "[I] {title}" - comment_issue: "Re: [I] {title}" - close_issue: "Re: [I] {title}" - catchall: "[GH] {title}" - new_discussion: "[D] {title}" - edit_discussion: "Re: [D] {title}" - close_discussion: "Re: [D] {title}" - close_discussion_with_comment: "Re: [D] {title}" - reopen_discussion: "Re: [D] {title}" - new_comment_discussion: "Re: [D] {title}" - edit_comment_discussion: "Re: [D] {title}" - delete_comment_discussion: "Re: [D] {title}" + new_pr: "[PR] {title} ({repository})" + close_pr: "Re: [PR] {title} ({repository})" + comment_pr: "Re: [PR] {title} ({repository})" + diffcomment: "Re: [PR] {title} ({repository})" + merge_pr: "Re: [PR] {title} ({repository})" + new_issue: "[I] {title} ({repository})" + comment_issue: "Re: [I] {title} ({repository})" + close_issue: "Re: [I] {title} ({repository})" + catchall: "[GH] {title} ({repository})" + new_discussion: "[D] {title} ({repository})" + edit_discussion: "Re: [D] {title} ({repository})" + close_discussion: "Re: [D] {title} ({repository})" + close_discussion_with_comment: "Re: [D] {title} ({repository})" + reopen_discussion: "Re: [D] {title} ({repository})" + new_comment_discussion: "Re: [D] {title} ({repository})" + edit_comment_discussion: "Re: [D] {title} ({repository})" + delete_comment_discussion: "Re: [D] {title} ({repository})" notifications: commits: commits@druid.apache.org From c798d3fb2e840df5700615a0fb994f61acbed46e Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 28 Jun 2023 17:12:44 -0700 Subject: [PATCH 50/74] Fix flaky SqlStatementResourceTest. (#14498) Mocks generally have state and should not be static. In particular, the "Yielder" included in one of the mocks can only be iterated once, which made the test suite order-dependent. --- .../druid/msq/sql/SqlStatementResourceTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index 3f2e7288b62..b5d5addecd7 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -222,7 +222,7 @@ public class SqlStatementResourceTest extends MSQTestBase new Object[]{234, "foo1", "bar1"} ); - private static final MSQTaskReport MSQ_SELECT_TASK_REPORT = new MSQTaskReport( + private final MSQTaskReport selectTaskReport = new MSQTaskReport( FINISHED_SELECT_MSQ_QUERY, new MSQTaskReportPayload( new MSQStatusReport( @@ -315,10 +315,11 @@ public class SqlStatementResourceTest extends MSQTestBase ); private static final String FAILURE_MSG = "failure msg"; private static SqlStatementResource resource; - @Mock - private static OverlordClient overlordClient; - private static void setupMocks(OverlordClient indexingServiceClient) throws JsonProcessingException + @Mock + private OverlordClient overlordClient; + + private void setupMocks(OverlordClient indexingServiceClient) throws JsonProcessingException { Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) @@ -392,8 +393,7 @@ public class SqlStatementResourceTest extends MSQTestBase Mockito.when(indexingServiceClient.taskReportAsMap(FINISHED_SELECT_MSQ_QUERY)) .thenReturn(Futures.immediateFuture(mapper.readValue( - mapper.writeValueAsString(TaskReport.buildTaskReports( - MSQ_SELECT_TASK_REPORT)), + mapper.writeValueAsString(TaskReport.buildTaskReports(selectTaskReport)), new TypeReference>() { } From a6cabbe10f02d87f9299490a07dd437a4a559b8c Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 28 Jun 2023 21:27:11 -0700 Subject: [PATCH 51/74] SQL: Avoid "intervals" for non-table-based datasources. (#14336) In these other cases, stick to plain "filter". This simplifies lots of logic downstream, and doesn't hurt since we don't have intervals-specific optimizations outside of tables. Fixes an issue where we couldn't properly filter on a column from an external datasource if it was named __time. --- .../druid/msq/querykit/DataSourcePlan.java | 7 +- .../apache/druid/msq/exec/MSQInsertTest.java | 4 +- .../org/apache/druid/query/DataSource.java | 2 +- .../apache/druid/query/UnionQueryRunner.java | 2 +- .../query/planning/DataSourceAnalysis.java | 44 ++++++---- .../planning/DataSourceAnalysisTest.java | 48 +++++++---- .../server/ClientQuerySegmentWalker.java | 4 +- .../server/TestClusterQuerySegmentWalker.java | 4 +- .../druid/sql/calcite/rel/DruidQuery.java | 85 ++++++++++++------- 9 files changed, 127 insertions(+), 73 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java index d6a21fc1338..95f5eae7bb4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java @@ -512,8 +512,11 @@ public class DataSourcePlan * interval {@link Intervals#ETERNITY}. If not, throw an {@link UnsupportedOperationException}. * * Anywhere this appears is a place that we do not support using the "intervals" parameter of a query - * (i.e., {@link org.apache.druid.query.BaseQuery#getQuerySegmentSpec()}) for time filtering. Ideally, - * we'd support this everywhere it appears, but we can get away without it for now. + * (i.e., {@link org.apache.druid.query.BaseQuery#getQuerySegmentSpec()}) for time filtering. + * + * We don't need to support this for anything that is not {@link DataSourceAnalysis#isTableBased()}, because + * the SQL layer avoids "intervals" in other cases. See + * {@link org.apache.druid.sql.calcite.rel.DruidQuery#canUseIntervalFiltering(DataSource)}. */ private static void checkQuerySegmentSpecIsEternity( final DataSource dataSource, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index 08a5629b2c0..fde14fa3016 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -173,7 +173,9 @@ public class MSQInsertTest extends MSQTestBase + " '{\"type\": \"json\"}',\n" + " '[{\"name\": \"__time\", \"type\": \"long\"}, {\"name\": \"flags\", \"type\": \"string\"}]'\n" + " )\n" - + ") PARTITIONED BY day") + + ")\n" + + "WHERE __time > TIMESTAMP '1999-01-01 00:00:00'\n" + + "PARTITIONED BY day") .setQueryContext(context) .setExpectedResultRows(expectedRows) .setExpectedDataSource("foo1") diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index ce589bc9f82..59a81843963 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -96,7 +96,7 @@ public interface DataSource * query stack. For example, {@link QueryDataSource} must be executed first and substituted with its results. * * @see DataSourceAnalysis#isConcreteBased() which uses this - * @see DataSourceAnalysis#isConcreteTableBased() which uses this + * @see DataSourceAnalysis#isConcreteAndTableBased() which uses this */ boolean isConcrete(); diff --git a/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java b/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java index 07b2947c27f..aeb3897e644 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java @@ -52,7 +52,7 @@ public class UnionQueryRunner implements QueryRunner final DataSourceAnalysis analysis = query.getDataSource().getAnalysis(); - if (analysis.isConcreteTableBased() && analysis.getBaseUnionDataSource().isPresent()) { + if (analysis.isConcreteAndTableBased() && analysis.getBaseUnionDataSource().isPresent()) { // Union of tables. final UnionDataSource unionDataSource = analysis.getBaseUnionDataSource().get(); diff --git a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java index 1a6cf49f914..f17ab6aec23 100644 --- a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java +++ b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java @@ -110,7 +110,7 @@ public class DataSourceAnalysis /** * If {@link #getBaseDataSource()} is a {@link TableDataSource}, returns it. Otherwise, returns an empty Optional. * - * Note that this can return empty even if {@link #isConcreteTableBased()} is true. This happens if the base + * Note that this can return empty even if {@link #isConcreteAndTableBased()} is true. This happens if the base * datasource is a {@link UnionDataSource} of {@link TableDataSource}. */ public Optional getBaseTableDataSource() @@ -175,6 +175,7 @@ public class DataSourceAnalysis * Else this method creates a new analysis object with the base query provided in the input * * @param query the query to add to the analysis if the baseQuery is null + * * @return the existing analysis if it has non-null basequery, else a new one with the updated base query */ public DataSourceAnalysis maybeWithBaseQuery(Query query) @@ -204,25 +205,38 @@ public class DataSourceAnalysis } /** - * Returns true if this datasource is concrete-based (see {@link #isConcreteBased()}, and the base datasource is a - * {@link TableDataSource} or a {@link UnionDataSource} composed entirely of {@link TableDataSource} - * or an {@link UnnestDataSource} composed entirely of {@link TableDataSource} . This is an - * important property, because it corresponds to datasources that can be handled by Druid's distributed query stack. + * Returns whether this datasource is one of: + * + *

      + *
    • {@link TableDataSource}
    • + *
    • {@link UnionDataSource} composed entirely of {@link TableDataSource}
    • + *
    • {@link UnnestDataSource} composed entirely of {@link TableDataSource}
    • + *
    */ - public boolean isConcreteTableBased() + public boolean isTableBased() + { + return (baseDataSource instanceof TableDataSource + || (baseDataSource instanceof UnionDataSource && + baseDataSource.getChildren() + .stream() + .allMatch(ds -> ds instanceof TableDataSource)) + || (baseDataSource instanceof UnnestDataSource && + baseDataSource.getChildren() + .stream() + .allMatch(ds -> ds instanceof TableDataSource))); + } + + /** + * Returns true if this datasource is both (see {@link #isConcreteBased()} and {@link #isTableBased()}. + * This is an important property, because it corresponds to datasources that can be handled by Druid's distributed + * query stack. + */ + public boolean isConcreteAndTableBased() { // At the time of writing this comment, UnionDataSource children are required to be tables, so the instanceof // check is redundant. But in the future, we will likely want to support unions of things other than tables, // so check anyway for future-proofing. - return isConcreteBased() && (baseDataSource instanceof TableDataSource - || (baseDataSource instanceof UnionDataSource && - baseDataSource.getChildren() - .stream() - .allMatch(ds -> ds instanceof TableDataSource)) - || (baseDataSource instanceof UnnestDataSource && - baseDataSource.getChildren() - .stream() - .allMatch(ds -> ds instanceof TableDataSource))); + return isConcreteBased() && isTableBased(); } /** diff --git a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java index 88e1335c19d..1302e504dc9 100644 --- a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java +++ b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java @@ -65,7 +65,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = TABLE_FOO.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseUnionDataSource()); @@ -82,7 +83,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = unionDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(unionDataSource, analysis.getBaseDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.of(unionDataSource), analysis.getBaseUnionDataSource()); @@ -99,7 +101,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = queryDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseUnionDataSource()); @@ -120,7 +123,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = queryDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(unionDataSource, analysis.getBaseDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.of(unionDataSource), analysis.getBaseUnionDataSource()); @@ -139,7 +143,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = LOOKUP_LOOKYLOO.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isTableBased()); + Assert.assertFalse(analysis.isConcreteAndTableBased()); Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseUnionDataSource()); @@ -156,7 +161,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = queryDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isTableBased()); + Assert.assertFalse(analysis.isConcreteAndTableBased()); Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseUnionDataSource()); @@ -175,7 +181,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = INLINE.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isTableBased()); + Assert.assertFalse(analysis.isConcreteAndTableBased()); Assert.assertEquals(INLINE, analysis.getBaseDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseUnionDataSource()); @@ -212,7 +219,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = joinDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getJoinBaseTableFilter()); @@ -256,7 +264,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = joinDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); Assert.assertEquals(TrueDimFilter.instance(), analysis.getJoinBaseTableFilter().orElse(null)); @@ -307,7 +316,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = joinDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getJoinBaseTableFilter()); @@ -351,7 +361,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = joinDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); Assert.assertEquals(TrueDimFilter.instance(), analysis.getJoinBaseTableFilter().orElse(null)); @@ -381,7 +392,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = joinDataSource.getAnalysis(); Assert.assertFalse(analysis.isConcreteBased()); - Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertFalse(analysis.isConcreteAndTableBased()); Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); Assert.assertEquals(TrueDimFilter.instance(), analysis.getJoinBaseTableFilter().orElse(null)); Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); @@ -409,7 +421,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = joinDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getJoinBaseTableFilter()); Assert.assertEquals(Optional.of(unionDataSource), analysis.getBaseUnionDataSource()); @@ -444,7 +457,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = queryDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isTableBased()); + Assert.assertTrue(analysis.isConcreteAndTableBased()); Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); Assert.assertEquals(TrueDimFilter.instance(), analysis.getJoinBaseTableFilter().orElse(null)); Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); @@ -489,7 +503,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = joinDataSource.getAnalysis(); Assert.assertTrue(analysis.isConcreteBased()); - Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isTableBased()); + Assert.assertFalse(analysis.isConcreteAndTableBased()); Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseUnionDataSource()); @@ -518,7 +533,8 @@ public class DataSourceAnalysisTest final DataSourceAnalysis analysis = joinDataSource.getAnalysis(); Assert.assertFalse(analysis.isConcreteBased()); - Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isTableBased()); + Assert.assertFalse(analysis.isConcreteAndTableBased()); Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); Assert.assertEquals(Optional.empty(), analysis.getBaseUnionDataSource()); diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java index d66055229bc..8e79a6421f7 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java @@ -272,7 +272,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker // 2) Must be based on globally available data (so we have a copy here on the Broker). // 3) If there is an outer query, it must be handleable by the query toolchest (the local walker does not handle // subqueries on its own). - return analysis.isConcreteBased() && !analysis.isConcreteTableBased() && dataSourceFromQuery.isGlobal() + return analysis.isConcreteBased() && !analysis.isConcreteAndTableBased() && dataSourceFromQuery.isGlobal() && (!(dataSourceFromQuery instanceof QueryDataSource) || toolChest.canPerformSubquery(((QueryDataSource) dataSourceFromQuery).getQuery())); } @@ -290,7 +290,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker // 1) Must be based on a concrete table (the only shape the Druid cluster can handle). // 2) If there is an outer query, it must be handleable by the query toolchest (the cluster walker does not handle // subqueries on its own). - return analysis.isConcreteTableBased() + return analysis.isConcreteAndTableBased() && (!(dataSourceFromQuery instanceof QueryDataSource) || toolChest.canPerformSubquery(((QueryDataSource) dataSourceFromQuery).getQuery())); } diff --git a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java index 47a3ddc46d6..5fbfabbfcd7 100644 --- a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java @@ -95,7 +95,7 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker return (queryPlus, responseContext) -> { final DataSourceAnalysis analysis = queryPlus.getQuery().getDataSource().getAnalysis(); - if (!analysis.isConcreteTableBased()) { + if (!analysis.isConcreteAndTableBased()) { throw new ISE("Cannot handle datasource: %s", queryPlus.getQuery().getDataSource()); } @@ -121,7 +121,7 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker final DataSourceAnalysis analysis = dataSourceFromQuery.getAnalysis(); - if (!analysis.isConcreteTableBased()) { + if (!analysis.isConcreteAndTableBased()) { throw new ISE("Cannot handle datasource: %s", dataSourceFromQuery); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 3b80dda9dcc..11f7db35ac5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -774,7 +774,6 @@ public class DruidQuery * Returns a pair of DataSource and Filtration object created on the query filter. In case the, data source is * a join datasource, the datasource may be altered and left filter of join datasource may * be rid of time filters. - * TODO: should we optimize the base table filter just like we do with query filters */ @VisibleForTesting static Pair getFiltration( @@ -784,40 +783,58 @@ public class DruidQuery JoinableFactoryWrapper joinableFactoryWrapper ) { - if (!(dataSource instanceof JoinDataSource)) { - return Pair.of(dataSource, toFiltration(filter, virtualColumnRegistry)); + if (!canUseIntervalFiltering(dataSource)) { + return Pair.of(dataSource, toFiltration(filter, virtualColumnRegistry.getFullRowSignature(), false)); + } else if (dataSource instanceof JoinDataSource && ((JoinDataSource) dataSource).getLeftFilter() != null) { + final JoinDataSource joinDataSource = (JoinDataSource) dataSource; + + // If the join is left or inner, we can pull the intervals up to the query. This is done + // so that broker can prune the segments to query. + final Filtration leftFiltration = Filtration.create(joinDataSource.getLeftFilter()) + .optimize(virtualColumnRegistry.getFullRowSignature()); + + // Adds the intervals from the join left filter to query filtration + final Filtration queryFiltration = Filtration.create(filter, leftFiltration.getIntervals()) + .optimize(virtualColumnRegistry.getFullRowSignature()); + + final JoinDataSource newDataSource = JoinDataSource.create( + joinDataSource.getLeft(), + joinDataSource.getRight(), + joinDataSource.getRightPrefix(), + joinDataSource.getConditionAnalysis(), + joinDataSource.getJoinType(), + leftFiltration.getDimFilter(), + joinableFactoryWrapper + ); + + return Pair.of(newDataSource, queryFiltration); + } else { + return Pair.of(dataSource, toFiltration(filter, virtualColumnRegistry.getFullRowSignature(), true)); } - JoinDataSource joinDataSource = (JoinDataSource) dataSource; - if (joinDataSource.getLeftFilter() == null) { - return Pair.of(dataSource, toFiltration(filter, virtualColumnRegistry)); - } - //TODO: We should avoid promoting the time filter as interval for right outer and full outer joins. This is not - // done now as we apply the intervals to left base table today irrespective of the join type. - - // If the join is left or inner, we can pull the intervals up to the query. This is done - // so that broker can prune the segments to query. - Filtration leftFiltration = Filtration.create(joinDataSource.getLeftFilter()) - .optimize(virtualColumnRegistry.getFullRowSignature()); - // Adds the intervals from the join left filter to query filtration - Filtration queryFiltration = Filtration.create(filter, leftFiltration.getIntervals()) - .optimize(virtualColumnRegistry.getFullRowSignature()); - - - JoinDataSource newDataSource = JoinDataSource.create( - joinDataSource.getLeft(), - joinDataSource.getRight(), - joinDataSource.getRightPrefix(), - joinDataSource.getConditionAnalysis(), - joinDataSource.getJoinType(), - leftFiltration.getDimFilter(), - joinableFactoryWrapper - ); - return Pair.of(newDataSource, queryFiltration); } - private static Filtration toFiltration(DimFilter filter, VirtualColumnRegistry virtualColumnRegistry) + /** + * Whether the given datasource can make use of "intervals" based filtering. The is true for anything based on + * regular tables ({@link org.apache.druid.query.TableDataSource}). + */ + private static boolean canUseIntervalFiltering(final DataSource dataSource) { - return Filtration.create(filter).optimize(virtualColumnRegistry.getFullRowSignature()); + return dataSource.getAnalysis().isTableBased(); + } + + private static Filtration toFiltration( + final DimFilter filter, + final RowSignature rowSignature, + final boolean useIntervals + ) + { + final Filtration filtration = Filtration.create(filter); + + if (useIntervals) { + return filtration.optimize(rowSignature); + } else { + return filtration.optimizeFilterOnly(rowSignature); + } } /** @@ -837,7 +854,7 @@ public class DruidQuery return true; } - if (dataSource.getAnalysis().isConcreteTableBased()) { + if (dataSource.getAnalysis().isConcreteAndTableBased()) { // Always OK: queries on concrete tables (regular Druid datasources) use segment-based storage adapters // (IncrementalIndex or QueryableIndex). These clip query interval to data interval, making wide query // intervals safer. They do not have special checks for granularity and interval safety. @@ -1430,7 +1447,9 @@ public class DruidQuery } if (!plannerContext.featureAvailable(EngineFeature.SCAN_ORDER_BY_NON_TIME) && !orderByColumns.isEmpty()) { - if (orderByColumns.size() > 1 || !ColumnHolder.TIME_COLUMN_NAME.equals(orderByColumns.get(0).getColumnName())) { + if (orderByColumns.size() > 1 + || orderByColumns.stream() + .anyMatch(orderBy -> !orderBy.getColumnName().equals(ColumnHolder.TIME_COLUMN_NAME))) { // We cannot handle this ordering, but we encounter this ordering as part of the exploration of the volcano // planner, which means that the query that we are looking right now might only be doing this as one of the // potential branches of exploration rather than being a semantic requirement of the query itself. So, it is From e10e35aa2ca953e65f1337a08800e0a29aaefd44 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 29 Jun 2023 13:47:57 -0700 Subject: [PATCH 52/74] Add REGEXP_REPLACE function. (#14460) * Add REGEXP_REPLACE function. Replaces all instances of a pattern with a replacement string. * Fixes. * Improve test coverage. * Adjust behavior. --- docs/querying/math-expr.md | 1 + docs/querying/sql-functions.md | 9 + docs/querying/sql-scalar.md | 1 + .../expression/RegexpReplaceExprMacro.java | 157 ++++++++++++ .../RegexpReplaceExprMacroTest.java | 238 ++++++++++++++++++ .../query/expression/TestExprMacroTable.java | 2 + .../apache/druid/guice/ExpressionModule.java | 2 + .../RegexpReplaceOperatorConversion.java | 69 +++++ .../calcite/planner/DruidOperatorTable.java | 3 + .../calcite/expression/ExpressionsTest.java | 53 ++++ website/.spelling | 1 + 11 files changed, 536 insertions(+) create mode 100644 processing/src/main/java/org/apache/druid/query/expression/RegexpReplaceExprMacro.java create mode 100644 processing/src/test/java/org/apache/druid/query/expression/RegexpReplaceExprMacroTest.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpReplaceOperatorConversion.java diff --git a/docs/querying/math-expr.md b/docs/querying/math-expr.md index 2f50d4102cf..8d558f4ceb9 100644 --- a/docs/querying/math-expr.md +++ b/docs/querying/math-expr.md @@ -84,6 +84,7 @@ The following built-in functions are available. |parse_long|parse_long(string[, radix]) parses a string as a long with the given radix, or 10 (decimal) if a radix is not provided.| |regexp_extract|regexp_extract(expr, pattern[, index]) applies a regular expression pattern and extracts a capture group index, or null if there is no match. If index is unspecified or zero, returns the substring that matched the pattern. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern.| |regexp_like|regexp_like(expr, pattern) returns whether `expr` matches regular expression `pattern`. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern. | +|regexp_replace|regexp_replace(expr, pattern, replacement) replaces all instances of a regular expression pattern with a given replacement string. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern.| |contains_string|contains_string(expr, string) returns whether `expr` contains `string` as a substring. This method is case-sensitive.| |icontains_string|contains_string(expr, string) returns whether `expr` contains `string` as a substring. This method is case-insensitive.| |replace|replace(expr, pattern, replacement) replaces pattern with replacement| diff --git a/docs/querying/sql-functions.md b/docs/querying/sql-functions.md index 80532f2aca6..3e4cf711dcd 100644 --- a/docs/querying/sql-functions.md +++ b/docs/querying/sql-functions.md @@ -1141,6 +1141,15 @@ Applies a regular expression to the string expression and returns the _n_th matc Returns true or false signifying whether the regular expression finds a match in the string expression. +## REGEXP_REPLACE + +`REGEXP_REPLACE(, , )` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + +Replaces all occurrences of a regular expression in a string expression with a replacement string. The replacement +string may refer to capture groups using `$1`, `$2`, etc. + ## REPEAT `REPEAT(, [])` diff --git a/docs/querying/sql-scalar.md b/docs/querying/sql-scalar.md index ffdac7b7a21..a741c1ff8ba 100644 --- a/docs/querying/sql-scalar.md +++ b/docs/querying/sql-scalar.md @@ -103,6 +103,7 @@ String functions accept strings, and return a type appropriate to the function. |`POSITION(needle IN haystack [FROM fromIndex])`|Returns the index of `needle` within `haystack`, with indexes starting from 1. The search will begin at `fromIndex`, or 1 if `fromIndex` is not specified. If `needle` is not found, returns 0.| |`REGEXP_EXTRACT(expr, pattern, [index])`|Apply regular expression `pattern` to `expr` and extract a capture group, or `NULL` if there is no match. If index is unspecified or zero, returns the first substring that matched the pattern. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern. Note: when `druid.generic.useDefaultValueForNull = true`, it is not possible to differentiate an empty-string match from a non-match (both will return `NULL`).| |`REGEXP_LIKE(expr, pattern)`|Returns whether `expr` matches regular expression `pattern`. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern. Similar to [`LIKE`](sql-operators.md#logical-operators), but uses regexps instead of LIKE patterns. Especially useful in WHERE clauses.| +|`REGEXP_REPLACE(expr, pattern, replacement)`|Replaces all occurrences of regular expression `pattern` within `expr` with `replacement`. The replacement string may refer to capture groups using `$1`, `$2`, etc. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern.| |`CONTAINS_STRING(expr, str)`|Returns true if the `str` is a substring of `expr`.| |`ICONTAINS_STRING(expr, str)`|Returns true if the `str` is a substring of `expr`. The match is case-insensitive.| |`REPLACE(expr, pattern, replacement)`|Replaces pattern with replacement in `expr`, and returns the result.| diff --git a/processing/src/main/java/org/apache/druid/query/expression/RegexpReplaceExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/RegexpReplaceExprMacro.java new file mode 100644 index 00000000000..cf4ecf83a77 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/expression/RegexpReplaceExprMacro.java @@ -0,0 +1,157 @@ +/* + * 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.druid.query.expression; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.math.expr.Expr; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.math.expr.ExpressionType; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class RegexpReplaceExprMacro implements ExprMacroTable.ExprMacro +{ + private static final String FN_NAME = "regexp_replace"; + + @Override + public String name() + { + return FN_NAME; + } + + @Override + public Expr apply(final List args) + { + validationHelperCheckArgumentCount(args, 3); + + if (args.stream().skip(1).allMatch(Expr::isLiteral)) { + return new RegexpReplaceExpr(args); + } else { + return new RegexpReplaceDynamicExpr(args); + } + } + + abstract class BaseRegexpReplaceExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr + { + public BaseRegexpReplaceExpr(final List args) + { + super(FN_NAME, args); + } + + @Nullable + @Override + public ExpressionType getOutputType(InputBindingInspector inspector) + { + return ExpressionType.STRING; + } + + @Override + public Expr visit(Shuttle shuttle) + { + return shuttle.visit(apply(shuttle.visitAll(args))); + } + } + + /** + * Expr when pattern and replacement are literals. + */ + class RegexpReplaceExpr extends BaseRegexpReplaceExpr + { + private final Expr arg; + private final Pattern pattern; + private final String replacement; + + private RegexpReplaceExpr(List args) + { + super(args); + + final Expr patternExpr = args.get(1); + final Expr replacementExpr = args.get(2); + + if (!ExprUtils.isStringLiteral(patternExpr) + && !(patternExpr.isLiteral() && patternExpr.getLiteralValue() == null)) { + throw validationFailed("pattern must be a string literal"); + } + + if (!ExprUtils.isStringLiteral(replacementExpr) + && !(replacementExpr.isLiteral() && replacementExpr.getLiteralValue() == null)) { + throw validationFailed("replacement must be a string literal"); + } + + final String patternString = NullHandling.nullToEmptyIfNeeded((String) patternExpr.getLiteralValue()); + + this.arg = args.get(0); + this.pattern = patternString != null ? Pattern.compile(patternString) : null; + this.replacement = NullHandling.nullToEmptyIfNeeded((String) replacementExpr.getLiteralValue()); + } + + @Nonnull + @Override + public ExprEval eval(final ObjectBinding bindings) + { + if (pattern == null || replacement == null) { + return ExprEval.of(null); + } + + final String s = NullHandling.nullToEmptyIfNeeded(arg.eval(bindings).asString()); + + if (s == null) { + return ExprEval.of(null); + } else { + final Matcher matcher = pattern.matcher(s); + final String retVal = matcher.replaceAll(replacement); + return ExprEval.of(retVal); + } + } + } + + /** + * Expr when pattern and replacement are dynamic (not literals). + */ + class RegexpReplaceDynamicExpr extends BaseRegexpReplaceExpr + { + private RegexpReplaceDynamicExpr(List args) + { + super(args); + } + + @Nonnull + @Override + public ExprEval eval(final ObjectBinding bindings) + { + final String s = NullHandling.nullToEmptyIfNeeded(args.get(0).eval(bindings).asString()); + final String pattern = NullHandling.nullToEmptyIfNeeded(args.get(1).eval(bindings).asString()); + final String replacement = NullHandling.nullToEmptyIfNeeded(args.get(2).eval(bindings).asString()); + + if (s == null || pattern == null || replacement == null) { + return ExprEval.of(null); + } else { + final Matcher matcher = Pattern.compile(pattern).matcher(s); + final String retVal = matcher.replaceAll(replacement); + return ExprEval.of(retVal); + } + } + } +} diff --git a/processing/src/test/java/org/apache/druid/query/expression/RegexpReplaceExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/RegexpReplaceExprMacroTest.java new file mode 100644 index 00000000000..f0e3f3c843d --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/expression/RegexpReplaceExprMacroTest.java @@ -0,0 +1,238 @@ +/* + * 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.druid.query.expression; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.math.expr.InputBindings; +import org.junit.Assert; +import org.junit.Test; + +public class RegexpReplaceExprMacroTest extends MacroTestBase +{ + public RegexpReplaceExprMacroTest() + { + super(new RegexpReplaceExprMacro()); + } + + @Test + public void testErrorZeroArguments() + { + expectException(IllegalArgumentException.class, "Function[regexp_replace] requires 3 arguments"); + eval("regexp_replace()", InputBindings.nilBindings()); + } + + @Test + public void testErrorFourArguments() + { + expectException(IllegalArgumentException.class, "Function[regexp_replace] requires 3 arguments"); + eval("regexp_replace('a', 'b', 'c', 'd')", InputBindings.nilBindings()); + } + + @Test + public void testErrorNonStringPattern() + { + expectException(IllegalArgumentException.class, "Function[regexp_replace] pattern must be a string literal"); + eval( + "regexp_replace(a, 1, 'x')", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo") + ); + } + + @Test + public void testErrorNonStringReplacement() + { + expectException(IllegalArgumentException.class, "Function[regexp_replace] replacement must be a string literal"); + eval( + "regexp_replace(a, 'x', 1)", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo") + ); + } + + @Test + public void testNullPattern() + { + final ExprEval result = eval( + "regexp_replace(a, null, 'x')", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo") + ); + + if (NullHandling.sqlCompatible()) { + Assert.assertNull(result.value()); + } else { + Assert.assertEquals("xfxoxox", result.value()); + } + } + + @Test + public void testNoMatch() + { + final ExprEval result = eval( + "regexp_replace(a, 'f.x', 'beep')", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo") + ); + Assert.assertEquals("foo", result.value()); + } + + @Test + public void testEmptyStringPattern() + { + final ExprEval result = eval( + "regexp_replace(a, '', 'x')", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo") + ); + Assert.assertEquals("xfxoxox", result.value()); + } + + @Test + public void testMultiLinePattern() + { + final ExprEval result = eval( + "regexp_replace(a, '^foo\\\\nbar$', 'xxx')", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo\nbar") + ); + Assert.assertEquals("xxx", result.value()); + } + + @Test + public void testMultiLinePatternNoMatch() + { + final ExprEval result = eval( + "regexp_replace(a, '^foo\\\\nbar$', 'xxx')", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "foo\nbarz") + ); + Assert.assertEquals("foo\nbarz", result.value()); + } + + @Test + public void testNullPatternOnEmptyString() + { + final ExprEval result = eval( + "regexp_replace(a, null, 'x')", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "") + ); + + if (NullHandling.sqlCompatible()) { + Assert.assertNull(result.value()); + } else { + Assert.assertEquals("x", result.value()); + } + } + + @Test + public void testEmptyStringPatternOnEmptyString() + { + final ExprEval result = eval( + "regexp_replace(a, '', 'x')", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "") + ); + Assert.assertEquals("x", result.value()); + } + + @Test + public void testEmptyStringPatternOnEmptyStringDynamic() + { + final ExprEval result = eval( + "regexp_replace(a, pattern, replacement)", + InputBindings.forInputSuppliers( + ImmutableMap.of( + "a", InputBindings.inputSupplier(ExpressionType.STRING, () -> ""), + "pattern", InputBindings.inputSupplier(ExpressionType.STRING, () -> ""), + "replacement", InputBindings.inputSupplier(ExpressionType.STRING, () -> "x") + ) + ) + ); + Assert.assertEquals("x", result.value()); + } + + @Test + public void testNullPatternOnNull() + { + final ExprEval result = eval("regexp_replace(a, null, 'x')", InputBindings.nilBindings()); + + if (NullHandling.sqlCompatible()) { + Assert.assertNull(result.value()); + } else { + Assert.assertEquals("x", result.value()); + } + } + + @Test + public void testNullPatternOnNullDynamic() + { + final ExprEval result = eval( + "regexp_replace(a, pattern, replacement)", + InputBindings.forInputSuppliers( + ImmutableMap.of("replacement", InputBindings.inputSupplier(ExpressionType.STRING, () -> "x")) + ) + ); + + if (NullHandling.sqlCompatible()) { + Assert.assertNull(result.value()); + } else { + Assert.assertEquals("x", result.value()); + } + } + + @Test + public void testEmptyStringPatternOnNull() + { + final ExprEval result = eval("regexp_replace(a, '', 'x')", InputBindings.nilBindings()); + + if (NullHandling.sqlCompatible()) { + Assert.assertNull(result.value()); + } else { + Assert.assertEquals("x", result.value()); + } + } + + @Test + public void testUrlIdReplacement() + { + final ExprEval result = eval( + "regexp_replace(regexp_replace(a, '\\\\?(.*)$', ''), '/(\\\\w+)(?=/|$)', '/*')", + InputBindings.forInputSupplier("a", ExpressionType.STRING, () -> "http://example.com/path/to?query") + ); + + Assert.assertEquals("http://example.com/*/*", result.value()); + } + + @Test + public void testUrlIdReplacementDynamic() + { + final ExprEval result = eval( + "regexp_replace(regexp_replace(a, pattern1, replacement1), pattern2, replacement2)", + InputBindings.forInputSuppliers( + ImmutableMap + .builder() + .put("a", InputBindings.inputSupplier(ExpressionType.STRING, () -> "http://example.com/path/to?query")) + .put("pattern1", InputBindings.inputSupplier(ExpressionType.STRING, () -> "\\?(.*)$")) + .put("pattern2", InputBindings.inputSupplier(ExpressionType.STRING, () -> "/(\\w+)(?=/|$)")) + .put("replacement1", InputBindings.inputSupplier(ExpressionType.STRING, () -> "")) + .put("replacement2", InputBindings.inputSupplier(ExpressionType.STRING, () -> "/*")) + .build() + ) + ); + + Assert.assertEquals("http://example.com/*/*", result.value()); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/expression/TestExprMacroTable.java b/processing/src/test/java/org/apache/druid/query/expression/TestExprMacroTable.java index cadb24dc980..93ba9878f78 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/TestExprMacroTable.java +++ b/processing/src/test/java/org/apache/druid/query/expression/TestExprMacroTable.java @@ -42,7 +42,9 @@ public class TestExprMacroTable extends ExprMacroTable new IPv4AddressParseExprMacro(), new IPv4AddressStringifyExprMacro(), new LikeExprMacro(), + new RegexpLikeExprMacro(), new RegexpExtractExprMacro(), + new RegexpReplaceExprMacro(), new TimestampCeilExprMacro(), new TimestampExtractExprMacro(), new TimestampFloorExprMacro(), diff --git a/server/src/main/java/org/apache/druid/guice/ExpressionModule.java b/server/src/main/java/org/apache/druid/guice/ExpressionModule.java index 3008ba8fb05..59f9ddb6c84 100644 --- a/server/src/main/java/org/apache/druid/guice/ExpressionModule.java +++ b/server/src/main/java/org/apache/druid/guice/ExpressionModule.java @@ -36,6 +36,7 @@ import org.apache.druid.query.expression.LikeExprMacro; import org.apache.druid.query.expression.NestedDataExpressions; import org.apache.druid.query.expression.RegexpExtractExprMacro; import org.apache.druid.query.expression.RegexpLikeExprMacro; +import org.apache.druid.query.expression.RegexpReplaceExprMacro; import org.apache.druid.query.expression.TimestampCeilExprMacro; import org.apache.druid.query.expression.TimestampExtractExprMacro; import org.apache.druid.query.expression.TimestampFloorExprMacro; @@ -57,6 +58,7 @@ public class ExpressionModule implements Module .add(LikeExprMacro.class) .add(RegexpExtractExprMacro.class) .add(RegexpLikeExprMacro.class) + .add(RegexpReplaceExprMacro.class) .add(ContainsExprMacro.class) .add(CaseInsensitiveContainsExprMacro.class) .add(TimestampCeilExprMacro.class) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpReplaceOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpReplaceOperatorConversion.java new file mode 100644 index 00000000000..134d72e9146 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpReplaceOperatorConversion.java @@ -0,0 +1,69 @@ +/* + * 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.druid.sql.calcite.expression.builtin; + +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.expression.DruidExpression; +import org.apache.druid.sql.calcite.expression.OperatorConversions; +import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; +import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.PlannerContext; + +public class RegexpReplaceOperatorConversion implements SqlOperatorConversion +{ + private static final SqlFunction SQL_FUNCTION = OperatorConversions + .operatorBuilder("REGEXP_REPLACE") + .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) + .requiredOperands(3) + .returnTypeNullable(SqlTypeName.VARCHAR) + .functionCategory(SqlFunctionCategory.STRING) + .build(); + + @Override + public SqlFunction calciteOperator() + { + return SQL_FUNCTION; + } + + @Override + public DruidExpression toDruidExpression( + final PlannerContext plannerContext, + final RowSignature rowSignature, + final RexNode rexNode + ) + { + return OperatorConversions.convertCall( + plannerContext, + rowSignature, + rexNode, + arguments -> DruidExpression.ofFunctionCall( + Calcites.getColumnTypeForRelDataType(rexNode.getType()), + StringUtils.toLowerCase(SQL_FUNCTION.getName()), + arguments + ) + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java index b373e8a01b8..eac61b8b440 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java @@ -100,6 +100,7 @@ import org.apache.druid.sql.calcite.expression.builtin.RPadOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RTrimOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RegexpExtractOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RegexpLikeOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.RegexpReplaceOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ReinterpretOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RepeatOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ReverseOperatorConversion; @@ -200,6 +201,7 @@ public class DruidOperatorTable implements SqlOperatorTable .add(new PositionOperatorConversion()) .add(new RegexpExtractOperatorConversion()) .add(new RegexpLikeOperatorConversion()) + .add(new RegexpReplaceOperatorConversion()) .add(new RTrimOperatorConversion()) .add(new ParseLongOperatorConversion()) .add(new StringFormatOperatorConversion()) @@ -533,6 +535,7 @@ public class DruidOperatorTable implements SqlOperatorTable * than prefix/suffix/binary syntax as function syntax. * * @param syntax The SqlSyntax value to be checked. + * * @return {@code true} if the syntax is valid for a function, {@code false} otherwise. */ public static boolean isFunctionSyntax(final SqlSyntax syntax) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java index 3fbd517e1cb..6610951f07d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java @@ -50,6 +50,7 @@ import org.apache.druid.sql.calcite.expression.builtin.ParseLongOperatorConversi import org.apache.druid.sql.calcite.expression.builtin.RPadOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RegexpExtractOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RegexpLikeOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.RegexpReplaceOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RepeatOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ReverseOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RightOperatorConversion; @@ -390,6 +391,58 @@ public class ExpressionsTest extends ExpressionTestBase ); } + @Test + public void testRegexpReplace() + { + testHelper.testExpressionString( + new RegexpReplaceOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("x(.)"), + testHelper.makeLiteral("z") + ), + makeExpression("regexp_replace(\"s\",'x(.)','z')"), + "foo" + ); + + testHelper.testExpressionString( + new RegexpReplaceOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("(o)"), + testHelper.makeLiteral("z") + ), + makeExpression("regexp_replace(\"s\",'(o)','z')"), + "fzz" + ); + + testHelper.testExpressionString( + new RegexpReplaceOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeCall( + SqlStdOperatorTable.CONCAT, + testHelper.makeLiteral("Z"), + testHelper.makeInputRef("s") + ), + testHelper.makeLiteral("Zf(.)"), + testHelper.makeLiteral("z") + ), + makeExpression("regexp_replace(concat('Z',\"s\"),'Zf(.)','z')"), + "zo" + ); + + testHelper.testExpressionString( + new RegexpReplaceOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("f(.)"), + testHelper.makeLiteral("$1") + ), + makeExpression("regexp_replace(\"s\",'f(.)','$1')"), + "oo" + ); + } + @Test public void testRegexpLike() { diff --git a/website/.spelling b/website/.spelling index 997d387c762..63573f94a86 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1474,6 +1474,7 @@ nvl parse_long regexp_extract regexp_like +regexp_replace contains_string icontains_string result1 From 4b2d87336a56eafe6c28fc93f1040e737013671e Mon Sep 17 00:00:00 2001 From: Pranav Date: Thu, 29 Jun 2023 15:32:43 -0700 Subject: [PATCH 53/74] Add additional index on task table (#14470) --- .../druid/metadata/SQLMetadataConnector.java | 104 +++++++++++++++++- .../metadata/SQLMetadataConnectorTest.java | 57 ++++++++++ 2 files changed, 159 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 108128e09c2..5c8c9d39c26 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -19,10 +19,12 @@ package org.apache.druid.metadata; +import com.google.common.base.Joiner; import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import org.apache.commons.dbcp2.BasicDataSource; import org.apache.commons.dbcp2.BasicDataSourceFactory; import org.apache.druid.java.util.common.ISE; @@ -51,8 +53,10 @@ import java.sql.SQLRecoverableException; import java.sql.SQLTransientException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Properties; +import java.util.Set; public abstract class SQLMetadataConnector implements MetadataStorageConnector { @@ -377,10 +381,22 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector + " PRIMARY KEY (id)\n" + ")", tableName, getPayloadType(), getCollation() - ), - StringUtils.format("CREATE INDEX idx_%1$s_active_created_date ON %1$s(active, created_date)", tableName) + ) ) ); + final Set createdIndexSet = getIndexOnTable(tableName); + createIndex( + tableName, + StringUtils.format("idx_%1$s_active_created_date", tableName), + ImmutableList.of("active", "created_date"), + createdIndexSet + ); + createIndex( + tableName, + StringUtils.format("idx_%1$s_datasource_active", tableName), + ImmutableList.of("datasource", "active"), + createdIndexSet + ); } private void alterEntryTable(final String tableName) @@ -830,4 +846,88 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector log.warn(e, "Exception while deleting records from table"); } } + + /** + * Get the Set of the index on given table + * + * @param tableName name of the table to fetch the index map + * @return Set of the uppercase index names, returns empty set if table does not exist + */ + public Set getIndexOnTable(String tableName) + { + Set res = new HashSet<>(); + try { + retryWithHandle(new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + DatabaseMetaData databaseMetaData = handle.getConnection().getMetaData(); + // Fetch the index for given table + ResultSet resultSet = databaseMetaData.getIndexInfo( + null, + null, + StringUtils.toUpperCase(tableName), + false, + false + ); + while (resultSet.next()) { + String indexName = resultSet.getString("INDEX_NAME"); + if (org.apache.commons.lang.StringUtils.isNotBlank(indexName)) { + res.add(StringUtils.toUpperCase(indexName)); + } + } + return null; + } + }); + } + catch (Exception e) { + log.error(e, "Exception while listing the index on table %s ", tableName); + } + return ImmutableSet.copyOf(res); + } + + /** + * create index on the table with retry if not already exist, to be called after createTable + * + * @param tableName Name of the table to create index on + * @param indexName case-insensitive string index name, it helps to check the existing index on table + * @param indexCols List of columns to be indexed on + * @param createdIndexSet + */ + public void createIndex( + final String tableName, + final String indexName, + final List indexCols, + final Set createdIndexSet + ) + { + try { + retryWithHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) + { + if (!createdIndexSet.contains(StringUtils.toUpperCase(indexName))) { + String indexSQL = StringUtils.format( + "CREATE INDEX %1$s ON %2$s(%3$s)", + indexName, + tableName, + Joiner.on(",").join(indexCols) + ); + log.info("Creating Index on Table [%s], sql: [%s] ", tableName, indexSQL); + handle.execute(indexSQL); + } else { + log.info("Index [%s] on Table [%s] already exists", indexName, tableName); + } + return null; + } + } + ); + } + catch (Exception e) { + log.error(e, StringUtils.format("Exception while creating index on table [%s]", tableName)); + } + } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index d9df47920da..4dba35ca84b 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -22,6 +22,8 @@ package org.apache.druid.metadata; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.apache.commons.dbcp2.BasicDataSource; import org.apache.druid.java.util.common.StringUtils; import org.junit.Assert; @@ -41,8 +43,11 @@ import java.sql.SQLTransientException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; public class SQLMetadataConnectorTest @@ -107,6 +112,58 @@ public class SQLMetadataConnectorTest } } + @Test + public void testIndexCreationOnTaskTable() + { + final String entryType = tablesConfig.getTaskEntryType(); + String entryTableName = tablesConfig.getEntryTable(entryType); + connector.createTaskTables(); + Set createdIndexSet = connector.getIndexOnTable(entryTableName); + Set expectedIndexSet = Sets.newHashSet( + StringUtils.format("idx_%1$s_active_created_date", entryTableName), + StringUtils.format("idx_%1$s_datasource_active", entryTableName) + ).stream().map(StringUtils::toUpperCase).collect(Collectors.toSet()); + + for (String expectedIndex : expectedIndexSet) { + Assert.assertTrue( + StringUtils.format("Failed to find the expected Index %s on entry table", expectedIndex), + createdIndexSet.contains(expectedIndex) + ); + } + connector.createTaskTables(); + dropTable(entryTableName); + } + + @Test + public void testCreateIndexOnNoTable() + { + String tableName = "noTable"; + try { + connector.createIndex( + tableName, + "some_string", + Lists.newArrayList("a", "b"), + new HashSet<>() + ); + } + catch (Exception e) { + Assert.fail("Index creation should never throw an exception"); + } + } + + @Test + public void testGeIndexOnNoTable() + { + String tableName = "noTable"; + try { + Set res = connector.getIndexOnTable(tableName); + Assert.assertEquals(0, res.size()); + } + catch (Exception e) { + Assert.fail("getIndexOnTable should never throw an exception"); + } + } + @Test public void testInsertOrUpdate() { From 67fbd8e7fcf71e8a0dc716b612317226e02bff88 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 30 Jun 2023 12:45:55 -0700 Subject: [PATCH 54/74] Add "stringEncoding" parameter to DataSketches HLL. (#11201) * Add "stringEncoding" parameter to DataSketches HLL. Builds on the concept from #11172 and adds a way to feed HLL sketches with UTF-8 bytes. This must be an option rather than always-on, because prior to this patch, HLL sketches used UTF-16LE encoding when hashing strings. To remain compatible with sketch images created prior to this patch -- which matters during rolling updates and when reading sketches that have been written to segments -- we must keep UTF-16LE as the default. Not currently documented, because I'm not yet sure how best to expose this functionality to users. I think the first place would be in the SQL layer: we could have it automatically select UTF-8 or UTF-16LE when building sketches at query time. We need to be careful about this, though, because UTF-8 isn't always faster. Sometimes, like for the results of expressions, UTF-16LE is faster. I expect we will sort this out in future patches. * Fix benchmark. * Fix style issues, improve test coverage. * Put round back, to make IT updates easier. * Fix test. * Fix issue with filtered aggregators and add test. * Use DS native update(ByteBuffer) method. Improve test coverage. * Add another suppression. * Fix ITAutoCompactionTest. * Update benchmarks. * Updates. * Fix conflict. * Adjustments. --- .../benchmark/DataSketchesHllBenchmark.java | 1 + .../druid/benchmark/query/SqlBenchmark.java | 39 +- codestyle/spotbugs-exclude.xml | 1 + .../hll/HllSketchAggregatorFactory.java | 44 +- .../hll/HllSketchBuildAggregator.java | 55 +- .../hll/HllSketchBuildAggregatorFactory.java | 38 +- .../hll/HllSketchBuildBufferAggregator.java | 24 +- .../HllSketchBuildColumnProcessorFactory.java | 110 ++++ .../datasketches/hll/HllSketchBuildUtil.java | 103 ++++ .../hll/HllSketchBuildVectorAggregator.java | 68 ++- .../datasketches/hll/HllSketchHolder.java | 6 + .../hll/HllSketchMergeAggregatorFactory.java | 17 +- .../hll/HllSketchMergeVectorAggregator.java | 7 +- .../datasketches/hll/HllSketchModule.java | 2 + ...ketchApproxCountDistinctSqlAggregator.java | 8 +- ...hApproxCountDistinctUtf8SqlAggregator.java | 84 +++ .../hll/sql/HllSketchBaseSqlAggregator.java | 16 +- .../hll/sql/HllSketchObjectSqlAggregator.java | 3 +- .../DoubleHllSketchBuildVectorProcessor.java | 74 +++ .../vector/HllSketchBuildVectorProcessor.java | 35 ++ .../HllSketchBuildVectorProcessorFactory.java | 94 ++++ .../LongHllSketchBuildVectorProcessor.java | 74 +++ ...ueStringHllSketchBuildVectorProcessor.java | 93 ++++ .../ObjectHllSketchBuildVectorProcessor.java | 87 +++ ...ueStringHllSketchBuildVectorProcessor.java | 82 +++ .../hll/HllSketchAggregatorFactoryTest.java | 45 +- .../hll/HllSketchAggregatorTest.java | 128 +++-- .../HllSketchBuildAggregatorFactoryTest.java | 110 ++++ .../hll/HllSketchBuildAggregatorTest.java | 135 ----- .../hll/HllSketchBuildUtilTest.java | 294 +++++++++++ .../HllSketchMergeAggregatorFactoryTest.java | 124 ++++- ...HllSketchToEstimatePostAggregatorTest.java | 1 + .../hll/sql/HllSketchSqlAggregatorTest.java | 497 ++++++++++-------- .../duty/ITAutoCompactionTest.java | 12 +- .../java/util/common/StringEncoding.java | 65 +++ ...codingDefaultUTF16LEJsonIncludeFilter.java | 39 ++ ...ngDefaultUTF16LEJsonIncludeFilterTest.java | 37 ++ .../java/util/common/StringEncodingTest.java | 57 ++ 38 files changed, 2171 insertions(+), 538 deletions(-) create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildColumnProcessorFactory.java create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildUtil.java create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctUtf8SqlAggregator.java create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/DoubleHllSketchBuildVectorProcessor.java create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessor.java create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessorFactory.java create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/LongHllSketchBuildVectorProcessor.java create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/MultiValueStringHllSketchBuildVectorProcessor.java create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/ObjectHllSketchBuildVectorProcessor.java create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/SingleValueStringHllSketchBuildVectorProcessor.java create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactoryTest.java delete mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorTest.java create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildUtilTest.java create mode 100644 processing/src/main/java/org/apache/druid/java/util/common/StringEncoding.java create mode 100644 processing/src/main/java/org/apache/druid/java/util/common/StringEncodingDefaultUTF16LEJsonIncludeFilter.java create mode 100644 processing/src/test/java/org/apache/druid/java/util/common/StringEncodingDefaultUTF16LEJsonIncludeFilterTest.java create mode 100644 processing/src/test/java/org/apache/druid/java/util/common/StringEncodingTest.java diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/DataSketchesHllBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/DataSketchesHllBenchmark.java index bed30c85c64..28dc5db9905 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/DataSketchesHllBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DataSketchesHllBenchmark.java @@ -65,6 +65,7 @@ public class DataSketchesHllBenchmark null, null, null, + null, false ); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index ba9debb2881..1172d823e02 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -32,8 +32,10 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchApproxCountDistinctSqlAggregator; +import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchApproxCountDistinctUtf8SqlAggregator; import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchApproxQuantileSqlAggregator; import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchObjectSqlAggregator; +import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchApproxCountDistinctSqlAggregator; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -405,26 +407,35 @@ public class SqlBenchmark "SELECT * FROM foo WHERE dimSequential IN ('1', '2', '3', '4', '5', '10', '11', '20', '21', '23', '40', '50', '64', '70', '100')", "SELECT * FROM foo WHERE dimSequential > '10' AND dimSequential < '8500'", "SELECT dimSequential, dimZipf, SUM(sumLongSequential) FROM foo WHERE dimSequential IN ('1', '2', '3', '4', '5', '10', '11', '20', '21', '23', '40', '50', '64', '70', '100') GROUP BY 1, 2", - "SELECT dimSequential, dimZipf, SUM(sumLongSequential) FROM foo WHERE dimSequential > '10' AND dimSequential < '8500' GROUP BY 1, 2" - + "SELECT dimSequential, dimZipf, SUM(sumLongSequential) FROM foo WHERE dimSequential > '10' AND dimSequential < '8500' GROUP BY 1, 2", + // 28, 29, 30, 31: Approximate count distinct of strings + "SELECT APPROX_COUNT_DISTINCT_BUILTIN(dimZipf) FROM foo", + "SELECT APPROX_COUNT_DISTINCT_DS_HLL(dimZipf) FROM foo", + "SELECT APPROX_COUNT_DISTINCT_DS_HLL_UTF8(dimZipf) FROM foo", + "SELECT APPROX_COUNT_DISTINCT_DS_THETA(dimZipf) FROM foo" ); @Param({"5000000"}) private int rowsPerSegment; - @Param({"false", "force"}) + // Can be "false", "true", or "force" + @Param({"force"}) private String vectorize; - @Param({"none", "front-coded-4", "front-coded-16"}) + + // Can be "none" or "front-coded-N" + @Param({"none", "front-coded-4"}) private String stringEncoding; - @Param({"4", "5", "6", "7", "8", "10", "11", "12", "19", "21", "22", "23", "26", "27"}) + @Param({"28", "29", "30", "31"}) private String query; - @Param({STORAGE_MMAP, STORAGE_FRAME_ROW, STORAGE_FRAME_COLUMNAR}) + // Can be STORAGE_MMAP, STORAGE_FRAME_ROW, or STORAGE_FRAME_COLUMNAR + @Param({STORAGE_MMAP}) private String storageType; private SqlEngine engine; + @Nullable private PlannerFactory plannerFactory; private final Closer closer = Closer.create(); @@ -520,13 +531,19 @@ public class SqlBenchmark try { final Set extractionOperators = new HashSet<>(); extractionOperators.add(CalciteTests.INJECTOR.getInstance(QueryLookupOperatorConversion.class)); - final Set aggregators = new HashSet<>(); - aggregators.add(CalciteTests.INJECTOR.getInstance(DoublesSketchApproxQuantileSqlAggregator.class)); - aggregators.add(CalciteTests.INJECTOR.getInstance(DoublesSketchObjectSqlAggregator.class)); final ApproxCountDistinctSqlAggregator countDistinctSqlAggregator = new ApproxCountDistinctSqlAggregator(new HllSketchApproxCountDistinctSqlAggregator()); - aggregators.add(new CountSqlAggregator(countDistinctSqlAggregator)); - aggregators.add(countDistinctSqlAggregator); + final Set aggregators = new HashSet<>( + ImmutableList.of( + new DoublesSketchApproxQuantileSqlAggregator(), + new DoublesSketchObjectSqlAggregator(), + new HllSketchApproxCountDistinctSqlAggregator(), + new HllSketchApproxCountDistinctUtf8SqlAggregator(), + new ThetaSketchApproxCountDistinctSqlAggregator(), + new CountSqlAggregator(countDistinctSqlAggregator), + countDistinctSqlAggregator + ) + ); return new DruidOperatorTable(aggregators, extractionOperators); } catch (Exception e) { diff --git a/codestyle/spotbugs-exclude.xml b/codestyle/spotbugs-exclude.xml index 79a6faf61c5..19da270192d 100644 --- a/codestyle/spotbugs-exclude.xml +++ b/codestyle/spotbugs-exclude.xml @@ -45,6 +45,7 @@ + diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java index 2b65e5a40e2..4bc734dc005 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java @@ -25,6 +25,8 @@ import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.hll.TgtHllType; import org.apache.datasketches.hll.Union; import org.apache.druid.jackson.DefaultTrueJsonIncludeFilter; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.java.util.common.StringEncodingDefaultUTF16LEJsonIncludeFilter; import org.apache.druid.query.aggregation.AggregateCombiner; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.ObjectAggregateCombiner; @@ -47,6 +49,7 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory public static final boolean DEFAULT_SHOULD_FINALIZE = true; public static final int DEFAULT_LG_K = 12; public static final TgtHllType DEFAULT_TGT_HLL_TYPE = TgtHllType.HLL_4; + public static final StringEncoding DEFAULT_STRING_ENCODING = StringEncoding.UTF16LE; static final Comparator COMPARATOR = Comparator.nullsFirst(Comparator.comparingDouble(HllSketchHolder::getEstimate)); @@ -55,6 +58,7 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory private final String fieldName; private final int lgK; private final TgtHllType tgtHllType; + private final StringEncoding stringEncoding; private final boolean shouldFinalize; private final boolean round; @@ -63,6 +67,7 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory final String fieldName, @Nullable final Integer lgK, @Nullable final String tgtHllType, + @Nullable final StringEncoding stringEncoding, final Boolean shouldFinalize, final boolean round ) @@ -71,6 +76,7 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory this.fieldName = Objects.requireNonNull(fieldName); this.lgK = lgK == null ? DEFAULT_LG_K : lgK; this.tgtHllType = tgtHllType == null ? DEFAULT_TGT_HLL_TYPE : TgtHllType.valueOf(tgtHllType); + this.stringEncoding = stringEncoding == null ? DEFAULT_STRING_ENCODING : stringEncoding; this.shouldFinalize = shouldFinalize == null ? DEFAULT_SHOULD_FINALIZE : shouldFinalize; this.round = round; } @@ -100,6 +106,13 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory return tgtHllType.toString(); } + @JsonProperty + @JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = StringEncodingDefaultUTF16LEJsonIncludeFilter.class) + public StringEncoding getStringEncoding() + { + return stringEncoding; + } + @JsonProperty @JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = DefaultTrueJsonIncludeFilter.class) public boolean isShouldFinalize() @@ -121,14 +134,23 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory } /** - * This is a convoluted way to return a list of input field names this aggregator needs. - * Currently the returned factories are only used to obtain a field name by calling getName() method. + * Used by groupBy v1 to create a "transfer aggregator". + * + * {@inheritDoc} */ @Override public List getRequiredColumns() { return Collections.singletonList( - new HllSketchBuildAggregatorFactory(fieldName, fieldName, lgK, tgtHllType.toString(), shouldFinalize, round) + new HllSketchBuildAggregatorFactory( + fieldName, + fieldName, + lgK, + tgtHllType.toString(), + stringEncoding, + shouldFinalize, + round + ) ); } @@ -228,6 +250,7 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory getName(), getLgK(), getTgtHllType(), + getStringEncoding(), isShouldFinalize(), isRound() ); @@ -236,8 +259,13 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory @Override public byte[] getCacheKey() { - return new CacheKeyBuilder(getCacheTypeId()).appendString(name).appendString(fieldName) - .appendInt(lgK).appendInt(tgtHllType.ordinal()).build(); + return new CacheKeyBuilder(getCacheTypeId()) + .appendString(name) + .appendString(fieldName) + .appendInt(lgK) + .appendInt(tgtHllType.ordinal()) + .appendCacheable(stringEncoding) + .build(); } @Override @@ -255,13 +283,14 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory && round == that.round && Objects.equals(name, that.name) && Objects.equals(fieldName, that.fieldName) - && tgtHllType == that.tgtHllType; + && tgtHllType == that.tgtHllType + && stringEncoding == that.stringEncoding; } @Override public int hashCode() { - return Objects.hash(name, fieldName, lgK, tgtHllType, shouldFinalize, round); + return Objects.hash(name, fieldName, lgK, tgtHllType, stringEncoding, shouldFinalize, round); } @Override @@ -272,6 +301,7 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory ", fieldName='" + fieldName + '\'' + ", lgK=" + lgK + ", tgtHllType=" + tgtHllType + + (stringEncoding != DEFAULT_STRING_ENCODING ? ", stringEncoding=" + stringEncoding : "") + (shouldFinalize != DEFAULT_SHOULD_FINALIZE ? ", shouldFinalize=" + shouldFinalize : "") + (round != DEFAULT_ROUND ? ", round=" + round : "") + '}'; diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregator.java index c2d848311f9..7a086b7257d 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregator.java @@ -21,12 +21,10 @@ package org.apache.druid.query.aggregation.datasketches.hll; import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.hll.TgtHllType; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.aggregation.Aggregator; -import org.apache.druid.segment.ColumnValueSelector; -import java.util.List; +import java.util.function.Consumer; +import java.util.function.Supplier; /** * This aggregator builds sketches from raw data. @@ -34,17 +32,16 @@ import java.util.List; */ public class HllSketchBuildAggregator implements Aggregator { - - private final ColumnValueSelector selector; + private final Consumer> processor; private HllSketch sketch; public HllSketchBuildAggregator( - final ColumnValueSelector selector, + final Consumer> processor, final int lgK, final TgtHllType tgtHllType ) { - this.selector = selector; + this.processor = processor; this.sketch = new HllSketch(lgK, tgtHllType); } @@ -54,15 +51,9 @@ public class HllSketchBuildAggregator implements Aggregator * See https://github.com/druid-io/druid/pull/3956 */ @Override - public void aggregate() + public synchronized void aggregate() { - final Object value = selector.getObject(); - if (value == null) { - return; - } - synchronized (this) { - updateSketch(sketch, value); - } + processor.accept(() -> sketch); } /* @@ -93,36 +84,4 @@ public class HllSketchBuildAggregator implements Aggregator { throw new UnsupportedOperationException("Not implemented"); } - - static void updateSketch(final HllSketch sketch, final Object value) - { - if (value instanceof Integer || value instanceof Long) { - sketch.update(((Number) value).longValue()); - } else if (value instanceof Float || value instanceof Double) { - sketch.update(((Number) value).doubleValue()); - } else if (value instanceof String) { - sketch.update(((String) value).toCharArray()); - } else if (value instanceof List) { - // noinspection rawtypes - for (Object entry : (List) value) { - if (entry != null) { - final String asString = entry.toString(); - if (!NullHandling.isNullOrEquivalent(asString)) { - sketch.update(asString); - } - } - } - } else if (value instanceof char[]) { - sketch.update((char[]) value); - } else if (value instanceof byte[]) { - sketch.update((byte[]) value); - } else if (value instanceof int[]) { - sketch.update((int[]) value); - } else if (value instanceof long[]) { - sketch.update((long[]) value); - } else { - throw new IAE("Unsupported type " + value.getClass()); - } - } - } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java index 226bda13245..2762d007b2e 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java @@ -24,20 +24,23 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.hll.TgtHllType; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; +import java.util.function.Consumer; +import java.util.function.Supplier; /** * This aggregator factory is for building sketches from raw data. @@ -53,11 +56,12 @@ public class HllSketchBuildAggregatorFactory extends HllSketchAggregatorFactory @JsonProperty("fieldName") final String fieldName, @JsonProperty("lgK") @Nullable final Integer lgK, @JsonProperty("tgtHllType") @Nullable final String tgtHllType, + @JsonProperty("stringEncoding") @Nullable final StringEncoding stringEncoding, @JsonProperty("shouldFinalize") final Boolean shouldFinalize, @JsonProperty("round") final boolean round ) { - super(name, fieldName, lgK, tgtHllType, shouldFinalize, round); + super(name, fieldName, lgK, tgtHllType, stringEncoding, shouldFinalize, round); } @@ -76,20 +80,37 @@ public class HllSketchBuildAggregatorFactory extends HllSketchAggregatorFactory @Override public Aggregator factorize(final ColumnSelectorFactory columnSelectorFactory) { - final ColumnValueSelector selector = columnSelectorFactory.makeColumnValueSelector(getFieldName()); validateInputs(columnSelectorFactory.getColumnCapabilities(getFieldName())); - return new HllSketchBuildAggregator(selector, getLgK(), TgtHllType.valueOf(getTgtHllType())); + + final Consumer> processor = ColumnProcessors.makeProcessor( + getFieldName(), + new HllSketchBuildColumnProcessorFactory(getStringEncoding()), + columnSelectorFactory + ); + + return new HllSketchBuildAggregator( + processor, + getLgK(), + TgtHllType.valueOf(getTgtHllType()) + ); } @Override public BufferAggregator factorizeBuffered(final ColumnSelectorFactory columnSelectorFactory) { - final ColumnValueSelector selector = columnSelectorFactory.makeColumnValueSelector(getFieldName()); validateInputs(columnSelectorFactory.getColumnCapabilities(getFieldName())); + + final Consumer> processor = ColumnProcessors.makeProcessor( + getFieldName(), + new HllSketchBuildColumnProcessorFactory(getStringEncoding()), + columnSelectorFactory + ); + return new HllSketchBuildBufferAggregator( - selector, + processor, getLgK(), TgtHllType.valueOf(getTgtHllType()), + getStringEncoding(), getMaxIntermediateSize() ); } @@ -104,11 +125,13 @@ public class HllSketchBuildAggregatorFactory extends HllSketchAggregatorFactory public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory) { validateInputs(selectorFactory.getColumnCapabilities(getFieldName())); - return new HllSketchBuildVectorAggregator( + + return HllSketchBuildVectorAggregator.create( selectorFactory, getFieldName(), getLgK(), TgtHllType.valueOf(getTgtHllType()), + getStringEncoding(), getMaxIntermediateSize() ); } @@ -131,6 +154,7 @@ public class HllSketchBuildAggregatorFactory extends HllSketchAggregatorFactory getFieldName(), getLgK(), getTgtHllType(), + getStringEncoding(), isShouldFinalize(), isRound() ); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildBufferAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildBufferAggregator.java index 7570e48d690..10cde4aa25d 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildBufferAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildBufferAggregator.java @@ -19,12 +19,15 @@ package org.apache.druid.query.aggregation.datasketches.hll; +import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.hll.TgtHllType; +import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.ColumnValueSelector; import java.nio.ByteBuffer; +import java.util.function.Consumer; +import java.util.function.Supplier; /** * This aggregator builds sketches from raw data. @@ -32,18 +35,21 @@ import java.nio.ByteBuffer; */ public class HllSketchBuildBufferAggregator implements BufferAggregator { - private final ColumnValueSelector selector; + private final Consumer> processor; private final HllSketchBuildBufferAggregatorHelper helper; + private final StringEncoding stringEncoding; public HllSketchBuildBufferAggregator( - final ColumnValueSelector selector, + final Consumer> processor, final int lgK, final TgtHllType tgtHllType, + final StringEncoding stringEncoding, final int size ) { - this.selector = selector; + this.processor = processor; this.helper = new HllSketchBuildBufferAggregatorHelper(lgK, tgtHllType, size); + this.stringEncoding = stringEncoding; } @Override @@ -55,12 +61,7 @@ public class HllSketchBuildBufferAggregator implements BufferAggregator @Override public void aggregate(final ByteBuffer buf, final int position) { - final Object value = selector.getObject(); - if (value == null) { - return; - } - - HllSketchBuildAggregator.updateSketch(helper.getSketchAtPosition(buf, position), value); + processor.accept(() -> helper.getSketchAtPosition(buf, position)); } @Override @@ -100,10 +101,11 @@ public class HllSketchBuildBufferAggregator implements BufferAggregator @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - inspector.visit("selector", selector); + inspector.visit("processor", processor); // lgK should be inspected because different execution paths exist in HllSketch.update() that is called from // @CalledFromHotLoop-annotated aggregate() depending on the lgK. // See https://github.com/apache/druid/pull/6893#discussion_r250726028 inspector.visit("lgK", helper.getLgK()); + inspector.visit("stringEncoding", stringEncoding); } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildColumnProcessorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildColumnProcessorFactory.java new file mode 100644 index 00000000000..d0823889578 --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildColumnProcessorFactory.java @@ -0,0 +1,110 @@ +/* + * 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.druid.query.aggregation.datasketches.hll; + +import org.apache.datasketches.hll.HllSketch; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.segment.BaseDoubleColumnValueSelector; +import org.apache.druid.segment.BaseFloatColumnValueSelector; +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnProcessorFactory; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.IndexedInts; + +import java.util.function.Consumer; +import java.util.function.Supplier; + +/** + * Scalar (non-vectorized) column processor factory. + */ +public class HllSketchBuildColumnProcessorFactory implements ColumnProcessorFactory>> +{ + private final StringEncoding stringEncoding; + + HllSketchBuildColumnProcessorFactory(StringEncoding stringEncoding) + { + this.stringEncoding = stringEncoding; + } + + @Override + public ColumnType defaultType() + { + return ColumnType.STRING; + } + + @Override + public Consumer> makeDimensionProcessor(DimensionSelector selector, boolean multiValue) + { + return sketch -> { + final IndexedInts row = selector.getRow(); + final int sz = row.size(); + + for (int i = 0; i < sz; i++) { + HllSketchBuildUtil.updateSketchWithDictionarySelector(sketch.get(), stringEncoding, selector, row.get(i)); + } + }; + } + + @Override + public Consumer> makeFloatProcessor(BaseFloatColumnValueSelector selector) + { + return sketch -> { + if (!selector.isNull()) { + // Important that this is *double* typed, since HllSketchBuildAggregator treats doubles and floats the same. + final double value = selector.getFloat(); + sketch.get().update(value); + } + }; + } + + @Override + public Consumer> makeDoubleProcessor(BaseDoubleColumnValueSelector selector) + { + return sketch -> { + if (!selector.isNull()) { + sketch.get().update(selector.getDouble()); + } + }; + } + + @Override + public Consumer> makeLongProcessor(BaseLongColumnValueSelector selector) + { + return sketch -> { + if (!selector.isNull()) { + sketch.get().update(selector.getLong()); + } + }; + } + + @Override + public Consumer> makeComplexProcessor(BaseObjectColumnValueSelector selector) + { + return sketch -> { + final Object o = selector.getObject(); + + if (o != null) { + HllSketchBuildUtil.updateSketch(sketch.get(), stringEncoding, o); + } + }; + } +} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildUtil.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildUtil.java new file mode 100644 index 00000000000..bcd4c4eb6d9 --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildUtil.java @@ -0,0 +1,103 @@ +/* + * 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.druid.query.aggregation.datasketches.hll; + +import org.apache.datasketches.hll.HllSketch; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.segment.DimensionDictionarySelector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.util.List; + +public class HllSketchBuildUtil +{ + public static void updateSketch(final HllSketch sketch, final StringEncoding stringEncoding, final Object value) + { + if (value instanceof Integer || value instanceof Long) { + sketch.update(((Number) value).longValue()); + } else if (value instanceof Float || value instanceof Double) { + sketch.update(((Number) value).doubleValue()); + } else if (value instanceof String) { + updateSketchWithString(sketch, stringEncoding, (String) value); + } else if (value instanceof List) { + // noinspection rawtypes + for (Object entry : (List) value) { + if (entry != null) { + updateSketchWithString(sketch, stringEncoding, entry.toString()); + } + } + } else if (value instanceof char[]) { + sketch.update((char[]) value); + } else if (value instanceof byte[]) { + sketch.update((byte[]) value); + } else if (value instanceof int[]) { + sketch.update((int[]) value); + } else if (value instanceof long[]) { + sketch.update((long[]) value); + } else { + throw new IAE("Unsupported type " + value.getClass()); + } + } + + public static void updateSketchWithDictionarySelector( + final HllSketch sketch, + final StringEncoding stringEncoding, + final DimensionDictionarySelector selector, + final int id + ) + { + if (stringEncoding == StringEncoding.UTF8 && selector.supportsLookupNameUtf8()) { + final ByteBuffer buf = selector.lookupNameUtf8(id); + + if (buf != null) { + sketch.update(buf); + } + } else { + updateSketchWithString(sketch, stringEncoding, selector.lookupName(id)); + } + } + + private static void updateSketchWithString( + final HllSketch sketch, + final StringEncoding stringEncoding, + @Nullable final String value + ) + { + if (NullHandling.isNullOrEquivalent(value)) { + return; + } + + switch (stringEncoding) { + case UTF8: + sketch.update(StringUtils.toUtf8(value)); + break; + case UTF16LE: + sketch.update(value.toCharArray()); + break; + default: + throw new UOE("Unsupported string encoding [%s]", stringEncoding); + } + } +} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildVectorAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildVectorAggregator.java index 275636bb68f..2901f282303 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildVectorAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildVectorAggregator.java @@ -20,35 +20,52 @@ package org.apache.druid.query.aggregation.datasketches.hll; import org.apache.datasketches.hll.TgtHllType; +import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.VectorAggregator; -import org.apache.druid.query.aggregation.datasketches.util.ToObjectVectorColumnProcessorFactory; +import org.apache.druid.query.aggregation.datasketches.hll.vector.HllSketchBuildVectorProcessor; +import org.apache.druid.query.aggregation.datasketches.hll.vector.HllSketchBuildVectorProcessorFactory; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.function.Supplier; public class HllSketchBuildVectorAggregator implements VectorAggregator { + private final HllSketchBuildVectorProcessor processor; private final HllSketchBuildBufferAggregatorHelper helper; - private final Supplier objectSupplier; - HllSketchBuildVectorAggregator( + private HllSketchBuildVectorAggregator( + final HllSketchBuildVectorProcessor processor, + final HllSketchBuildBufferAggregatorHelper helper + ) + { + this.processor = processor; + this.helper = helper; + } + + public static HllSketchBuildVectorAggregator create( final VectorColumnSelectorFactory columnSelectorFactory, final String column, final int lgK, final TgtHllType tgtHllType, + final StringEncoding stringEncoding, final int size ) { - this.helper = new HllSketchBuildBufferAggregatorHelper(lgK, tgtHllType, size); - this.objectSupplier = - ColumnProcessors.makeVectorProcessor( - column, - ToObjectVectorColumnProcessorFactory.INSTANCE, - columnSelectorFactory - ); + final HllSketchBuildBufferAggregatorHelper helper = new HllSketchBuildBufferAggregatorHelper( + lgK, + tgtHllType, + size + ); + + final HllSketchBuildVectorProcessor processor = ColumnProcessors.makeVectorProcessor( + column, + new HllSketchBuildVectorProcessorFactory(helper, stringEncoding), + columnSelectorFactory + ); + + return new HllSketchBuildVectorAggregator(processor, helper); } @Override @@ -58,36 +75,15 @@ public class HllSketchBuildVectorAggregator implements VectorAggregator } @Override - public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow) + public void aggregate(ByteBuffer buf, int position, int startRow, int endRow) { - final Object[] vector = objectSupplier.get(); - for (int i = startRow; i < endRow; i++) { - final Object value = vector[i]; - if (value != null) { - HllSketchBuildAggregator.updateSketch(helper.getSketchAtPosition(buf, position), value); - } - } + processor.aggregate(buf, position, startRow, endRow); } @Override - public void aggregate( - final ByteBuffer buf, - final int numRows, - final int[] positions, - @Nullable final int[] rows, - final int positionOffset - ) + public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset) { - final Object[] vector = objectSupplier.get(); - - for (int i = 0; i < numRows; i++) { - final Object o = vector[rows != null ? rows[i] : i]; - - if (o != null) { - final int position = positions[i] + positionOffset; - HllSketchBuildAggregator.updateSketch(helper.getSketchAtPosition(buf, position), o); - } - } + processor.aggregate(buf, numRows, positions, rows, positionOffset); } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchHolder.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchHolder.java index 81748bd6180..df0b884eaae 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchHolder.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchHolder.java @@ -162,4 +162,10 @@ public class HllSketchHolder return this; } } + + @Override + public String toString() + { + return "HllSketchHolder{" + (union != null ? union.toString() : sketch.toString()) + "}"; + } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java index 4645e82ea2c..833df8ab1a5 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.hll.TgtHllType; import org.apache.datasketches.hll.Union; +import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException; @@ -41,6 +42,12 @@ import javax.annotation.Nullable; /** * This aggregator factory is for merging existing sketches. * The input column must contain {@link HllSketch} + * + * Note: aggregators generated by this class do not directly use "stringEncoding", but it is part of this class + * anyway so we can preserve enough information to ensure that we are merging sketches in a valid way. (Sketches with + * incompatible string encodings cannot be merged meaningfully.) Currently, the only way this is exposed is through + * {@link #getMergingFactory}, which will throw {@link AggregatorFactoryNotMergeableException} if presented with + * two aggregators with two different encodings. */ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory { @@ -52,17 +59,20 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory @JsonProperty("fieldName") final String fieldName, @JsonProperty("lgK") @Nullable final Integer lgK, @JsonProperty("tgtHllType") @Nullable final String tgtHllType, + @JsonProperty("stringEncoding") @Nullable final StringEncoding stringEncoding, @JsonProperty("shouldFinalize") final Boolean shouldFinalize, @JsonProperty("round") final boolean round ) { - super(name, fieldName, lgK, tgtHllType, shouldFinalize, round); + super(name, fieldName, lgK, tgtHllType, stringEncoding, shouldFinalize, round); } @Override public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException { - if (other.getName().equals(this.getName()) && other instanceof HllSketchMergeAggregatorFactory) { + if (other.getName().equals(this.getName()) + && other instanceof HllSketchMergeAggregatorFactory + && getStringEncoding() == ((HllSketchMergeAggregatorFactory) other).getStringEncoding()) { HllSketchMergeAggregatorFactory castedOther = (HllSketchMergeAggregatorFactory) other; if (castedOther.isShouldFinalize() == isShouldFinalize()) { @@ -71,6 +81,7 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory getName(), Math.max(getLgK(), castedOther.getLgK()), getTgtHllType().compareTo(castedOther.getTgtHllType()) < 0 ? castedOther.getTgtHllType() : getTgtHllType(), + getStringEncoding(), isShouldFinalize(), isRound() || castedOther.isRound() ); @@ -145,9 +156,9 @@ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory getFieldName(), getLgK(), getTgtHllType(), + getStringEncoding(), isShouldFinalize(), isRound() ); } - } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeVectorAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeVectorAggregator.java index aee9bbd1672..8c7e214aa98 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeVectorAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeVectorAggregator.java @@ -85,6 +85,7 @@ public class HllSketchMergeVectorAggregator implements VectorAggregator final int positionOffset ) { + final WritableMemory mem = WritableMemory.writableWrap(buf, ByteOrder.LITTLE_ENDIAN); final Object[] vector = objectSupplier.get(); for (int i = 0; i < numRows; i++) { @@ -92,11 +93,7 @@ public class HllSketchMergeVectorAggregator implements VectorAggregator if (o != null) { final int position = positions[i] + positionOffset; - - final WritableMemory mem = WritableMemory.writableWrap(buf, ByteOrder.LITTLE_ENDIAN) - .writableRegion(position, helper.getSize()); - - final Union union = Union.writableWrap(mem); + final Union union = Union.writableWrap(mem.writableRegion(position, helper.getSize())); union.update(o.getSketch()); } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java index 68da26a94b4..ea2f11ca785 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java @@ -29,6 +29,7 @@ import org.apache.druid.guice.ExpressionModule; import org.apache.druid.initialization.DruidModule; import org.apache.druid.query.aggregation.datasketches.hll.sql.HllPostAggExprMacros; import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchApproxCountDistinctSqlAggregator; +import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchApproxCountDistinctUtf8SqlAggregator; import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchEstimateOperatorConversion; import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchEstimateWithErrorBoundsOperatorConversion; import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchObjectSqlAggregator; @@ -59,6 +60,7 @@ public class HllSketchModule implements DruidModule { registerSerde(); SqlBindings.addAggregator(binder, HllSketchApproxCountDistinctSqlAggregator.class); + SqlBindings.addAggregator(binder, HllSketchApproxCountDistinctUtf8SqlAggregator.class); SqlBindings.addAggregator(binder, HllSketchObjectSqlAggregator.class); SqlBindings.addOperatorConversion(binder, HllSketchEstimateOperatorConversion.class); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java index 29f7c819be1..757674d6aa6 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctSqlAggregator.java @@ -24,6 +24,7 @@ import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.type.InferTypes; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.sql.calcite.aggregation.Aggregation; @@ -48,7 +49,7 @@ public class HllSketchApproxCountDistinctSqlAggregator extends HllSketchBaseSqlA public HllSketchApproxCountDistinctSqlAggregator() { - super(true); + super(true, StringEncoding.UTF16LE); } @Override @@ -66,10 +67,7 @@ public class HllSketchApproxCountDistinctSqlAggregator extends HllSketchBaseSqlA { return Aggregation.create( Collections.singletonList(aggregatorFactory), - finalizeAggregations ? new FinalizingFieldAccessPostAggregator( - name, - aggregatorFactory.getName() - ) : null + finalizeAggregations ? new FinalizingFieldAccessPostAggregator(name, aggregatorFactory.getName()) : null ); } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctUtf8SqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctUtf8SqlAggregator.java new file mode 100644 index 00000000000..070fbd9f733 --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchApproxCountDistinctUtf8SqlAggregator.java @@ -0,0 +1,84 @@ +/* + * 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.druid.query.aggregation.datasketches.hll.sql; + +import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.type.InferTypes; +import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; +import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.SqlAggregator; +import org.apache.druid.sql.calcite.expression.OperatorConversions; + +import java.util.Collections; + +/** + * Like {@link HllSketchApproxCountDistinctSqlAggregator}, but uses {@link StringEncoding#UTF8} instead of + * {@link org.apache.druid.query.aggregation.datasketches.hll.HllSketchAggregatorFactory#DEFAULT_STRING_ENCODING}. + * + * Currently undocumented. Only accepts strings, not sketches. The purpose of this function is to allow us to experiment + * with UTF-8-based HLL counting while we figure out how the API should work. + * + * See https://github.com/apache/druid/pull/11201 for details. + */ +public class HllSketchApproxCountDistinctUtf8SqlAggregator + extends HllSketchBaseSqlAggregator + implements SqlAggregator +{ + public static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL_UTF8"; + private static final SqlAggFunction FUNCTION_INSTANCE = + OperatorConversions.aggregatorBuilder(NAME) + .operandNames("column", "lgK", "tgtHllType") + .operandTypes(SqlTypeFamily.STRING, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING) + .operandTypeInference(InferTypes.VARCHAR_1024) + .requiredOperandCount(1) + .literalOperands(1, 2) + .returnTypeNonNull(SqlTypeName.BIGINT) + .functionCategory(SqlFunctionCategory.NUMERIC) + .build(); + + public HllSketchApproxCountDistinctUtf8SqlAggregator() + { + super(true, StringEncoding.UTF8); + } + + @Override + public SqlAggFunction calciteFunction() + { + return FUNCTION_INSTANCE; + } + + @Override + protected Aggregation toAggregation( + String name, + boolean finalizeAggregations, + AggregatorFactory aggregatorFactory + ) + { + return Aggregation.create( + Collections.singletonList(aggregatorFactory), + finalizeAggregations ? new FinalizingFieldAccessPostAggregator(name, aggregatorFactory.getName()) : null + ); + } +} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java index 09f2609801d..c6dd3e7afa0 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java @@ -27,6 +27,7 @@ import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.datasketches.SketchQueryContext; import org.apache.druid.query.aggregation.datasketches.hll.HllSketchAggregatorFactory; @@ -53,10 +54,12 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator private static final boolean ROUND = true; private final boolean finalizeSketch; + private final StringEncoding stringEncoding; - protected HllSketchBaseSqlAggregator(boolean finalizeSketch) + protected HllSketchBaseSqlAggregator(boolean finalizeSketch, StringEncoding stringEncoding) { this.finalizeSketch = finalizeSketch; + this.stringEncoding = stringEncoding; } @Nullable @@ -137,6 +140,11 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator columnArg.getDirectColumn(), logK, tgtHllType, + + // For HllSketchMergeAggregatorFactory, stringEncoding is only advisory to aid in detection of mismatched + // merges. It does not affect the results of the aggregator. At this point in the code, we do not know what + // the input encoding of the original sketches was, so we set it to the default. + HllSketchAggregatorFactory.DEFAULT_STRING_ENCODING, finalizeSketch || SketchQueryContext.isFinalizeOuterSketches(plannerContext), ROUND ); @@ -169,6 +177,11 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator dimensionSpec.getOutputName(), logK, tgtHllType, + + // For HllSketchMergeAggregatorFactory, stringEncoding is only advisory to aid in detection of mismatched + // merges. It does not affect the results of the aggregator. At this point in the code, we do not know what + // the input encoding of the original sketches was, so we set it to the default. + HllSketchAggregatorFactory.DEFAULT_STRING_ENCODING, finalizeSketch || SketchQueryContext.isFinalizeOuterSketches(plannerContext), ROUND ); @@ -178,6 +191,7 @@ public abstract class HllSketchBaseSqlAggregator implements SqlAggregator dimensionSpec.getDimension(), logK, tgtHllType, + stringEncoding, finalizeSketch || SketchQueryContext.isFinalizeOuterSketches(plannerContext), ROUND ); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchObjectSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchObjectSqlAggregator.java index 8e695148b97..9d8ade636f1 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchObjectSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchObjectSqlAggregator.java @@ -25,6 +25,7 @@ import org.apache.calcite.sql.type.InferTypes; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchAggregatorFactory; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.OperatorConversions; @@ -47,7 +48,7 @@ public class HllSketchObjectSqlAggregator extends HllSketchBaseSqlAggregator imp public HllSketchObjectSqlAggregator() { - super(false); + super(false, HllSketchAggregatorFactory.DEFAULT_STRING_ENCODING); } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/DoubleHllSketchBuildVectorProcessor.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/DoubleHllSketchBuildVectorProcessor.java new file mode 100644 index 00000000000..7655a19992c --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/DoubleHllSketchBuildVectorProcessor.java @@ -0,0 +1,74 @@ +/* + * 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.druid.query.aggregation.datasketches.hll.vector; + +import org.apache.datasketches.hll.HllSketch; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildBufferAggregatorHelper; +import org.apache.druid.segment.vector.VectorValueSelector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +public class DoubleHllSketchBuildVectorProcessor implements HllSketchBuildVectorProcessor +{ + private final HllSketchBuildBufferAggregatorHelper helper; + private final VectorValueSelector selector; + + public DoubleHllSketchBuildVectorProcessor( + final HllSketchBuildBufferAggregatorHelper helper, + final VectorValueSelector selector + ) + { + this.helper = helper; + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position, int startRow, int endRow) + { + final double[] vector = selector.getDoubleVector(); + final boolean[] nullVector = selector.getNullVector(); + + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + + for (int i = startRow; i < endRow; i++) { + if (NullHandling.replaceWithDefault() || nullVector == null || !nullVector[i]) { + sketch.update(vector[i]); + } + } + } + + @Override + public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset) + { + final double[] vector = selector.getDoubleVector(); + final boolean[] nullVector = selector.getNullVector(); + + for (int i = 0; i < numRows; i++) { + final int idx = rows != null ? rows[i] : i; + if (NullHandling.replaceWithDefault() || nullVector == null || !nullVector[idx]) { + final int position = positions[i] + positionOffset; + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + sketch.update(vector[idx]); + } + } + } +} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessor.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessor.java new file mode 100644 index 00000000000..d5127b0dea5 --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessor.java @@ -0,0 +1,35 @@ +/* + * 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.druid.query.aggregation.datasketches.hll.vector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +/** + * Processor for {@link org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildVectorAggregator}. + * + * @see HllSketchBuildVectorProcessorFactory + */ +public interface HllSketchBuildVectorProcessor +{ + void aggregate(ByteBuffer buf, int position, int startRow, int endRow); + + void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset); +} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessorFactory.java new file mode 100644 index 00000000000..aac55a2e0b7 --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessorFactory.java @@ -0,0 +1,94 @@ +/* + * 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.druid.query.aggregation.datasketches.hll.vector; + +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildBufferAggregatorHelper; +import org.apache.druid.segment.VectorColumnProcessorFactory; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; +import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; +import org.apache.druid.segment.vector.VectorObjectSelector; +import org.apache.druid.segment.vector.VectorValueSelector; + +public class HllSketchBuildVectorProcessorFactory implements VectorColumnProcessorFactory +{ + private final HllSketchBuildBufferAggregatorHelper helper; + private final StringEncoding stringEncoding; + + public HllSketchBuildVectorProcessorFactory( + final HllSketchBuildBufferAggregatorHelper helper, + final StringEncoding stringEncoding + ) + { + this.helper = helper; + this.stringEncoding = stringEncoding; + } + + @Override + public HllSketchBuildVectorProcessor makeSingleValueDimensionProcessor( + ColumnCapabilities capabilities, + SingleValueDimensionVectorSelector selector + ) + { + return new SingleValueStringHllSketchBuildVectorProcessor(helper, stringEncoding, selector); + } + + @Override + public HllSketchBuildVectorProcessor makeMultiValueDimensionProcessor( + ColumnCapabilities capabilities, + MultiValueDimensionVectorSelector selector + ) + { + return new MultiValueStringHllSketchBuildVectorProcessor(helper, stringEncoding, selector); + } + + @Override + public HllSketchBuildVectorProcessor makeFloatProcessor(ColumnCapabilities capabilities, VectorValueSelector selector) + { + // No specialized "float" version, for consistency with HllSketchBuildAggregator#updateSketch (it treats floats + // and doubles identically). + return new DoubleHllSketchBuildVectorProcessor(helper, selector); + } + + @Override + public HllSketchBuildVectorProcessor makeDoubleProcessor( + ColumnCapabilities capabilities, + VectorValueSelector selector + ) + { + return new DoubleHllSketchBuildVectorProcessor(helper, selector); + } + + @Override + public HllSketchBuildVectorProcessor makeLongProcessor(ColumnCapabilities capabilities, VectorValueSelector selector) + { + return new LongHllSketchBuildVectorProcessor(helper, selector); + } + + @Override + public HllSketchBuildVectorProcessor makeObjectProcessor( + ColumnCapabilities capabilities, + VectorObjectSelector selector + ) + { + return new ObjectHllSketchBuildVectorProcessor(helper, stringEncoding, selector); + } +} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/LongHllSketchBuildVectorProcessor.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/LongHllSketchBuildVectorProcessor.java new file mode 100644 index 00000000000..398aef660b7 --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/LongHllSketchBuildVectorProcessor.java @@ -0,0 +1,74 @@ +/* + * 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.druid.query.aggregation.datasketches.hll.vector; + +import org.apache.datasketches.hll.HllSketch; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildBufferAggregatorHelper; +import org.apache.druid.segment.vector.VectorValueSelector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +public class LongHllSketchBuildVectorProcessor implements HllSketchBuildVectorProcessor +{ + private final HllSketchBuildBufferAggregatorHelper helper; + private final VectorValueSelector selector; + + public LongHllSketchBuildVectorProcessor( + final HllSketchBuildBufferAggregatorHelper helper, + final VectorValueSelector selector + ) + { + this.helper = helper; + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position, int startRow, int endRow) + { + final long[] vector = selector.getLongVector(); + final boolean[] nullVector = selector.getNullVector(); + + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + + for (int i = startRow; i < endRow; i++) { + if (NullHandling.replaceWithDefault() || nullVector == null || !nullVector[i]) { + sketch.update(vector[i]); + } + } + } + + @Override + public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset) + { + final long[] vector = selector.getLongVector(); + final boolean[] nullVector = selector.getNullVector(); + + for (int i = 0; i < numRows; i++) { + final int idx = rows != null ? rows[i] : i; + if (NullHandling.replaceWithDefault() || nullVector == null || !nullVector[idx]) { + final int position = positions[i] + positionOffset; + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + sketch.update(vector[idx]); + } + } + } +} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/MultiValueStringHllSketchBuildVectorProcessor.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/MultiValueStringHllSketchBuildVectorProcessor.java new file mode 100644 index 00000000000..074c18bbd6e --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/MultiValueStringHllSketchBuildVectorProcessor.java @@ -0,0 +1,93 @@ +/* + * 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.druid.query.aggregation.datasketches.hll.vector; + +import org.apache.datasketches.hll.HllSketch; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildBufferAggregatorHelper; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildUtil; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +public class MultiValueStringHllSketchBuildVectorProcessor implements HllSketchBuildVectorProcessor +{ + private final HllSketchBuildBufferAggregatorHelper helper; + private final StringEncoding stringEncoding; + private final MultiValueDimensionVectorSelector selector; + + public MultiValueStringHllSketchBuildVectorProcessor( + final HllSketchBuildBufferAggregatorHelper helper, + final StringEncoding stringEncoding, + final MultiValueDimensionVectorSelector selector + ) + { + this.helper = helper; + this.stringEncoding = stringEncoding; + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position, int startRow, int endRow) + { + final IndexedInts[] vector = selector.getRowVector(); + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + + for (int i = startRow; i < endRow; i++) { + final IndexedInts ids = vector[i]; + final int sz = ids.size(); + + for (int j = 0; j < sz; j++) { + HllSketchBuildUtil.updateSketchWithDictionarySelector( + sketch, + stringEncoding, + selector, + ids.get(j) + ); + } + } + } + + @Override + public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset) + { + final IndexedInts[] vector = selector.getRowVector(); + + for (int i = 0; i < numRows; i++) { + final int idx = rows != null ? rows[i] : i; + final int position = positions[i] + positionOffset; + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + + final IndexedInts ids = vector[idx]; + final int sz = ids.size(); + + for (int j = 0; j < sz; j++) { + HllSketchBuildUtil.updateSketchWithDictionarySelector( + sketch, + stringEncoding, + selector, + ids.get(j) + ); + } + } + } +} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/ObjectHllSketchBuildVectorProcessor.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/ObjectHllSketchBuildVectorProcessor.java new file mode 100644 index 00000000000..56eceb15f5c --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/ObjectHllSketchBuildVectorProcessor.java @@ -0,0 +1,87 @@ +/* + * 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.druid.query.aggregation.datasketches.hll.vector; + +import org.apache.datasketches.hll.HllSketch; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildBufferAggregatorHelper; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildUtil; +import org.apache.druid.segment.vector.VectorObjectSelector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +/** + * Processor that handles cases where string columns are presented as object selectors instead of dimension selectors. + */ +public class ObjectHllSketchBuildVectorProcessor implements HllSketchBuildVectorProcessor +{ + private final HllSketchBuildBufferAggregatorHelper helper; + private final StringEncoding stringEncoding; + private final VectorObjectSelector selector; + + public ObjectHllSketchBuildVectorProcessor( + final HllSketchBuildBufferAggregatorHelper helper, + final StringEncoding stringEncoding, + final VectorObjectSelector selector + ) + { + this.helper = helper; + this.stringEncoding = stringEncoding; + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position, int startRow, int endRow) + { + final Object[] vector = selector.getObjectVector(); + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + + for (int i = startRow; i < endRow; i++) { + if (vector[i] != null) { + HllSketchBuildUtil.updateSketch( + sketch, + stringEncoding, + vector[i] + ); + } + } + } + + @Override + public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset) + { + final Object[] vector = selector.getObjectVector(); + + for (int i = 0; i < numRows; i++) { + final int idx = rows != null ? rows[i] : i; + final int position = positions[i] + positionOffset; + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + + if (vector[idx] != null) { + HllSketchBuildUtil.updateSketch( + sketch, + stringEncoding, + vector[idx] + ); + } + } + } +} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/SingleValueStringHllSketchBuildVectorProcessor.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/SingleValueStringHllSketchBuildVectorProcessor.java new file mode 100644 index 00000000000..6c93b5669b8 --- /dev/null +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/SingleValueStringHllSketchBuildVectorProcessor.java @@ -0,0 +1,82 @@ +/* + * 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.druid.query.aggregation.datasketches.hll.vector; + +import org.apache.datasketches.hll.HllSketch; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildBufferAggregatorHelper; +import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildUtil; +import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +public class SingleValueStringHllSketchBuildVectorProcessor implements HllSketchBuildVectorProcessor +{ + private final HllSketchBuildBufferAggregatorHelper helper; + private final StringEncoding stringEncoding; + private final SingleValueDimensionVectorSelector selector; + + public SingleValueStringHllSketchBuildVectorProcessor( + final HllSketchBuildBufferAggregatorHelper helper, + final StringEncoding stringEncoding, + final SingleValueDimensionVectorSelector selector + ) + { + this.helper = helper; + this.stringEncoding = stringEncoding; + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position, int startRow, int endRow) + { + final int[] vector = selector.getRowVector(); + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + + for (int i = startRow; i < endRow; i++) { + HllSketchBuildUtil.updateSketchWithDictionarySelector( + sketch, + stringEncoding, + selector, + vector[i] + ); + } + } + + @Override + public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset) + { + final int[] vector = selector.getRowVector(); + + for (int i = 0; i < numRows; i++) { + final int idx = rows != null ? rows[i] : i; + final int position = positions[i] + positionOffset; + final HllSketch sketch = helper.getSketchAtPosition(buf, position); + + HllSketchBuildUtil.updateSketchWithDictionarySelector( + sketch, + stringEncoding, + selector, + vector[idx] + ); + } + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java index 12198fda6c9..2c4ff635faa 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java @@ -21,6 +21,7 @@ package org.apache.druid.query.aggregation.datasketches.hll; import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.hll.TgtHllType; +import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; import org.apache.druid.query.aggregation.Aggregator; @@ -54,6 +55,7 @@ public class HllSketchAggregatorFactoryTest private static final String FIELD_NAME = "fieldName"; private static final int LG_K = HllSketchAggregatorFactory.DEFAULT_LG_K; private static final String TGT_HLL_TYPE = TgtHllType.HLL_4.name(); + private static final StringEncoding STRING_ENCODING = StringEncoding.UTF16LE; private static final boolean ROUND = true; private static final double ESTIMATE = Math.PI; @@ -62,7 +64,7 @@ public class HllSketchAggregatorFactoryTest @Before public void setUp() { - target = new TestHllSketchAggregatorFactory(NAME, FIELD_NAME, LG_K, TGT_HLL_TYPE, ROUND); + target = new TestHllSketchAggregatorFactory(NAME, FIELD_NAME, LG_K, TGT_HLL_TYPE, STRING_ENCODING, ROUND); } @Test @@ -71,6 +73,12 @@ public class HllSketchAggregatorFactoryTest Assert.assertEquals(ROUND, target.isRound()); } + @Test + public void testStringEncoding() + { + Assert.assertEquals(STRING_ENCODING, target.getStringEncoding()); + } + @Test public void testGetRequiredColumns() { @@ -126,6 +134,7 @@ public class HllSketchAggregatorFactoryTest FIELD_NAME, LG_K, TGT_HLL_TYPE, + STRING_ENCODING, !ROUND ); Object actual = t.finalizeComputation(getMockSketch()); @@ -136,7 +145,9 @@ public class HllSketchAggregatorFactoryTest @Test public void testEqualsSameObject() { + //noinspection EqualsWithItself Assert.assertEquals(target, target); + Assert.assertArrayEquals(target.getCacheKey(), target.getCacheKey()); } @Test @@ -159,9 +170,11 @@ public class HllSketchAggregatorFactoryTest FIELD_NAME, LG_K, TGT_HLL_TYPE, + STRING_ENCODING, ROUND ); Assert.assertNotEquals(target, other); + Assert.assertFalse(Arrays.equals(target.getCacheKey(), other.getCacheKey())); } @Test @@ -172,9 +185,11 @@ public class HllSketchAggregatorFactoryTest FIELD_NAME + "-diff", LG_K, TGT_HLL_TYPE, + STRING_ENCODING, ROUND ); Assert.assertNotEquals(target, other); + Assert.assertFalse(Arrays.equals(target.getCacheKey(), other.getCacheKey())); } @Test @@ -185,9 +200,11 @@ public class HllSketchAggregatorFactoryTest FIELD_NAME, LG_K + 1, TGT_HLL_TYPE, + STRING_ENCODING, ROUND ); Assert.assertNotEquals(target, other); + Assert.assertFalse(Arrays.equals(target.getCacheKey(), other.getCacheKey())); } @Test @@ -198,9 +215,11 @@ public class HllSketchAggregatorFactoryTest FIELD_NAME, LG_K, TgtHllType.HLL_8.name(), + STRING_ENCODING, ROUND ); Assert.assertNotEquals(target, other); + Assert.assertFalse(Arrays.equals(target.getCacheKey(), other.getCacheKey())); } @Test @@ -211,9 +230,13 @@ public class HllSketchAggregatorFactoryTest FIELD_NAME, LG_K, TGT_HLL_TYPE, + STRING_ENCODING, !ROUND ); Assert.assertNotEquals(target, other); + + // Rounding does not affect per-segment results, so it does not affect cache key + Assert.assertArrayEquals(target.getCacheKey(), other.getCacheKey()); } @Test @@ -224,9 +247,11 @@ public class HllSketchAggregatorFactoryTest FIELD_NAME, LG_K, TGT_HLL_TYPE, + STRING_ENCODING, ROUND ); Assert.assertEquals(target, other); + Assert.assertArrayEquals(target.getCacheKey(), other.getCacheKey()); } @Test @@ -238,7 +263,7 @@ public class HllSketchAggregatorFactoryTest .collect(Collectors.toList()); for (Field field : toStringFields) { - if ("shouldFinalize".equals(field.getName())) { + if ("shouldFinalize".equals(field.getName()) || "stringEncoding".equals(field.getName())) { // Skip; not included in the toString if it has the default value. continue; } @@ -264,6 +289,7 @@ public class HllSketchAggregatorFactoryTest null, null, null, + null, false ), new HllSketchBuildAggregatorFactory( @@ -272,6 +298,7 @@ public class HllSketchAggregatorFactoryTest null, null, null, + null, true ), new HllSketchMergeAggregatorFactory( @@ -280,6 +307,7 @@ public class HllSketchAggregatorFactoryTest null, null, null, + null, false ), new HllSketchMergeAggregatorFactory( @@ -288,6 +316,7 @@ public class HllSketchAggregatorFactoryTest null, null, null, + null, true ) ) @@ -349,10 +378,11 @@ public class HllSketchAggregatorFactoryTest String fieldName, @Nullable Integer lgK, @Nullable String tgtHllType, + @Nullable StringEncoding stringEncoding, boolean round ) { - super(name, fieldName, lgK, tgtHllType, null, round); + super(name, fieldName, lgK, tgtHllType, stringEncoding, null, round); } @Override @@ -388,7 +418,14 @@ public class HllSketchAggregatorFactoryTest @Override public AggregatorFactory withName(String newName) { - return new TestHllSketchAggregatorFactory(newName, getFieldName(), getLgK(), getTgtHllType(), isRound()); + return new TestHllSketchAggregatorFactory( + newName, + getFieldName(), + getLgK(), + getTgtHllType(), + getStringEncoding(), + isRound() + ); } } } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java index 91f78ab40ff..b8acb0ce2c2 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; @@ -60,6 +61,7 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest private final AggregationTestHelper groupByHelper; private final AggregationTestHelper timeseriesHelper; private final QueryContexts.Vectorize vectorize; + private final StringEncoding stringEncoding; @Rule public final TemporaryFolder groupByFolder = new TemporaryFolder(); @@ -67,7 +69,7 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest @Rule public final TemporaryFolder timeseriesFolder = new TemporaryFolder(); - public HllSketchAggregatorTest(GroupByQueryConfig config, String vectorize) + public HllSketchAggregatorTest(GroupByQueryConfig config, String vectorize, StringEncoding stringEncoding) { HllSketchModule.registerSerde(); groupByHelper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( @@ -77,15 +79,20 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest new HllSketchModule().getJacksonModules(), timeseriesFolder ); this.vectorize = QueryContexts.Vectorize.fromString(vectorize); + this.stringEncoding = stringEncoding; } - @Parameterized.Parameters(name = "config = {0}, vectorize = {1}") + @Parameterized.Parameters(name = "groupByConfig = {0}, vectorize = {1}, stringEncoding = {2}") public static Collection constructorFeeder() { final List constructors = new ArrayList<>(); for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { - for (String vectorize : new String[]{"false", "true", "force"}) { - constructors.add(new Object[]{config, vectorize}); + for (String vectorize : new String[]{"false", "force"}) { + for (StringEncoding stringEncoding : StringEncoding.values()) { + if (!("v1".equals(config.getDefaultStrategy()) && "force".equals(vectorize))) { + constructors.add(new Object[]{config, vectorize, stringEncoding}); + } + } } } return constructors; @@ -100,11 +107,11 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest Arrays.asList("dim", "multiDim"), Arrays.asList("timestamp", "dim", "multiDim", "sketch") ), - buildAggregatorJson("HLLSketchMerge", "sketch", !ROUND), + buildAggregatorJson("HLLSketchMerge", "sketch", !ROUND, stringEncoding), 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount - buildGroupByQueryJson("HLLSketchMerge", "sketch", !ROUND) + 200, // maxRowCount + buildGroupByQueryJson("HLLSketchMerge", "sketch", !ROUND, stringEncoding) ); List results = seq.toList(); Assert.assertEquals(1, results.size()); @@ -120,17 +127,36 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest Arrays.asList("dim", "multiDim"), Arrays.asList("timestamp", "dim", "multiDim", "sketch") ); - final String aggregators = buildAggregatorJson("HLLSketchMerge", "sketch", !ROUND); + final String aggregators = + buildAggregatorJson("HLLSketchMerge", "sketch", !ROUND, HllSketchAggregatorFactory.DEFAULT_STRING_ENCODING); final int minTimestamp = 0; final Granularity gran = Granularities.NONE; final int maxRowCount = 10; final String queryJson = buildTimeseriesQueryJson("HLLSketchMerge", "sketch", !ROUND); File segmentDir1 = timeseriesFolder.newFolder(); - timeseriesHelper.createIndex(inputFile, parserJson, aggregators, segmentDir1, minTimestamp, gran, maxRowCount, true); + timeseriesHelper.createIndex( + inputFile, + parserJson, + aggregators, + segmentDir1, + minTimestamp, + gran, + maxRowCount, + true + ); File segmentDir2 = timeseriesFolder.newFolder(); - timeseriesHelper.createIndex(inputFile, parserJson, aggregators, segmentDir2, minTimestamp, gran, maxRowCount, true); + timeseriesHelper.createIndex( + inputFile, + parserJson, + aggregators, + segmentDir2, + minTimestamp, + gran, + maxRowCount, + true + ); Sequence seq = timeseriesHelper.runQueryOnSegments(Arrays.asList(segmentDir1, segmentDir2), queryJson); List results = seq.toList(); @@ -148,11 +174,11 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest Collections.singletonList("dim"), Arrays.asList("timestamp", "dim", "multiDim", "id") ), - buildAggregatorJson("HLLSketchBuild", "id", !ROUND), + buildAggregatorJson("HLLSketchBuild", "id", !ROUND, stringEncoding), 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount - buildGroupByQueryJson("HLLSketchMerge", "sketch", !ROUND) + 200, // maxRowCount + buildGroupByQueryJson("HLLSketchMerge", "sketch", !ROUND, stringEncoding) ); List results = seq.toList(); Assert.assertEquals(1, results.size()); @@ -169,10 +195,10 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest Collections.singletonList("dim"), Arrays.asList("timestamp", "dim", "multiDim", "id") ), - buildAggregatorJson("HLLSketchBuild", "id", !ROUND), + buildAggregatorJson("HLLSketchBuild", "id", !ROUND, stringEncoding), 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount + 200, // maxRowCount buildTimeseriesQueryJson("HLLSketchMerge", "sketch", !ROUND) ); List results = seq.toList(); @@ -193,8 +219,8 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest "[]", 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount - buildGroupByQueryJson("HLLSketchBuild", "id", !ROUND) + 200, // maxRowCount + buildGroupByQueryJson("HLLSketchBuild", "id", !ROUND, stringEncoding) ); List results = seq.toList(); Assert.assertEquals(1, results.size()); @@ -214,7 +240,7 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest "[]", 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount + 200, // maxRowCount buildTimeseriesQueryJson("HLLSketchBuild", "id", !ROUND) ); List results = seq.toList(); @@ -241,8 +267,8 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest metricSpec, 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount - buildGroupByQueryJson("HLLSketchBuild", "index_hll", !ROUND) + 200, // maxRowCount + buildGroupByQueryJson("HLLSketchMerge", "sketch", ROUND, stringEncoding) ); } catch (RuntimeException e) { @@ -263,8 +289,8 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest "[]", 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount - buildGroupByQueryJson("HLLSketchBuild", "multiDim", !ROUND) + 200, // maxRowCount + buildGroupByQueryJson("HLLSketchBuild", "multiDim", !ROUND, stringEncoding) ); List results = seq.toList(); Assert.assertEquals(1, results.size()); @@ -284,8 +310,8 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest "[]", 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount - buildGroupByQueryJson("HLLSketchBuild", "id", ROUND) + 200, // maxRowCount + buildGroupByQueryJson("HLLSketchBuild", "id", ROUND, stringEncoding) ); List results = seq.toList(); Assert.assertEquals(1, results.size()); @@ -302,11 +328,11 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest Arrays.asList("dim", "multiDim"), Arrays.asList("timestamp", "dim", "multiDim", "sketch") ), - buildAggregatorJson("HLLSketchMerge", "sketch", ROUND), + buildAggregatorJson("HLLSketchMerge", "sketch", ROUND, stringEncoding), 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount - buildGroupByQueryJson("HLLSketchMerge", "sketch", ROUND) + 200, // maxRowCount + buildGroupByQueryJson("HLLSketchMerge", "sketch", ROUND, stringEncoding) ); List results = seq.toList(); Assert.assertEquals(1, results.size()); @@ -323,17 +349,17 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest Arrays.asList("dim", "multiDim"), Arrays.asList("timestamp", "dim", "multiDim", "sketch") ), - buildAggregatorJson("HLLSketchMerge", "sketch", ROUND), + buildAggregatorJson("HLLSketchMerge", "sketch", ROUND, stringEncoding), 0, // minTimestamp Granularities.NONE, - 10, // maxRowCount + 200, // maxRowCount groupByHelper.getObjectMapper().writeValueAsString( GroupByQuery.builder() .setDataSource("test_datasource") .setGranularity(Granularities.ALL) .setInterval(Intervals.ETERNITY) .setAggregatorSpecs( - new HllSketchMergeAggregatorFactory("sketch", "sketch", null, null, null, false) + new HllSketchMergeAggregatorFactory("sketch", "sketch", null, null, null, null, false) ) .setPostAggregatorSpecs( ImmutableList.of( @@ -362,7 +388,8 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest ), new FieldAccessPostAggregator("f2", "sketch")), null, null - ) + ), + new FieldAccessPostAggregator("fieldAccess", "sketch") ) ) .build() @@ -378,6 +405,7 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest + " UB : 200.01008469948434\n" + " OutOfOrder Flag: false\n" + " Coupon Count : 200\n"; + List results = seq.toList(); Assert.assertEquals(1, results.size()); ResultRow row = results.get(0); @@ -430,13 +458,15 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest private static String buildAggregatorJson( String aggregationType, String aggregationFieldName, - boolean aggregationRound + boolean aggregationRound, + StringEncoding stringEncoding ) { Map aggregator = buildAggregatorObject( aggregationType, aggregationFieldName, - aggregationRound + aggregationRound, + stringEncoding ); return toJson(Collections.singletonList(aggregator)); } @@ -444,27 +474,32 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest private static Map buildAggregatorObject( String aggregationType, String aggregationFieldName, - boolean aggregationRound + boolean aggregationRound, + StringEncoding stringEncoding ) { - return ImmutableMap.of( - "type", aggregationType, - "name", "sketch", - "fieldName", aggregationFieldName, - "round", aggregationRound - ); + return ImmutableMap.builder() + .put("type", aggregationType) + .put("name", "sketch") + .put("fieldName", aggregationFieldName) + .put("round", aggregationRound) + .put("tgtHllType", "HLL_8") + .put("stringEncoding", stringEncoding.toString()) + .build(); } private String buildGroupByQueryJson( String aggregationType, String aggregationFieldName, - boolean aggregationRound + boolean aggregationRound, + StringEncoding stringEncoding ) { Map aggregation = buildAggregatorObject( aggregationType, aggregationFieldName, - aggregationRound + aggregationRound, + stringEncoding ); Map object = new ImmutableMap.Builder() .put("queryType", "groupBy") @@ -472,6 +507,12 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest .put("granularity", "ALL") .put("dimensions", Collections.emptyList()) .put("aggregations", Collections.singletonList(aggregation)) + .put( + "postAggregations", + Collections.singletonList( + ImmutableMap.of("type", "fieldAccess", "name", "sketch_raw", "fieldName", "sketch") + ) + ) .put("intervals", Collections.singletonList("2017-01-01T00:00:00.000Z/2017-01-31T00:00:00.000Z")) .put("context", ImmutableMap.of(QueryContexts.VECTORIZE_KEY, vectorize.toString())) .build(); @@ -487,7 +528,8 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest Map aggregation = buildAggregatorObject( aggregationType, aggregationFieldName, - aggregationRound + aggregationRound, + HllSketchAggregatorFactory.DEFAULT_STRING_ENCODING ); Map object = new ImmutableMap.Builder() .put("queryType", "timeseries") diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactoryTest.java new file mode 100644 index 00000000000..51ca671cd0d --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactoryTest.java @@ -0,0 +1,110 @@ +/* + * 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.druid.query.aggregation.datasketches.hll; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.datasketches.hll.TgtHllType; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class HllSketchBuildAggregatorFactoryTest +{ + private final ObjectMapper jsonMapper; + + public HllSketchBuildAggregatorFactoryTest() + { + this.jsonMapper = TestHelper.makeJsonMapper().copy(); + jsonMapper.registerModules(new HllSketchModule().getJacksonModules()); + } + + @Test + public void testSerde() throws IOException + { + final HllSketchBuildAggregatorFactory factory = new HllSketchBuildAggregatorFactory( + "foo", + "bar", + 18, + TgtHllType.HLL_8.name(), + StringEncoding.UTF8, + false, + true + ); + + final String serializedString = jsonMapper.writeValueAsString(factory); + + Assert.assertEquals( + "{\"type\":\"HLLSketchBuild\",\"name\":\"foo\",\"fieldName\":\"bar\",\"lgK\":18,\"tgtHllType\":\"HLL_8\"," + + "\"stringEncoding\":\"utf8\",\"shouldFinalize\":false,\"round\":true}", + serializedString + ); + + final AggregatorFactory factory2 = jsonMapper.readValue( + serializedString, + AggregatorFactory.class + ); + + Assert.assertEquals(factory, factory2); + } + + @Test + public void testSerdeWithDefaults() throws IOException + { + final HllSketchBuildAggregatorFactory factory = new HllSketchBuildAggregatorFactory( + "foo", + "bar", + null, + null, + null, + null, + false + ); + + final String serializedString = jsonMapper.writeValueAsString(factory); + + Assert.assertEquals( + "{\"type\":\"HLLSketchBuild\"," + + "\"name\":\"foo\"," + + "\"fieldName\":\"bar\"," + + "\"lgK\":12," + + "\"tgtHllType\":\"HLL_4\"" + + "}", + serializedString + ); + + final AggregatorFactory factory2 = jsonMapper.readValue( + serializedString, + AggregatorFactory.class + ); + + Assert.assertEquals(factory, factory2); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(HllSketchBuildAggregatorFactory.class).usingGetClass().verify(); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorTest.java deleted file mode 100644 index f1163407a91..00000000000 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorTest.java +++ /dev/null @@ -1,135 +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.druid.query.aggregation.datasketches.hll; - -import org.apache.datasketches.hll.HllSketch; -import org.apache.druid.testing.InitializedNullHandlingTest; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; - -/** - * Tests for {@link HllSketchBuildAggregator#updateSketch}. - * - * Tests of the aggregator generally should go in {@link HllSketchAggregatorTest} instead. - */ -public class HllSketchBuildAggregatorTest extends InitializedNullHandlingTest -{ - private final HllSketch sketch = new HllSketch(HllSketch.DEFAULT_LG_K); - - @Test - public void testUpdateSketchVariousNumbers() - { - updateSketch(1L, -2L, 1L, -2, 1L, 2.0, 2f, Double.doubleToLongBits(2.0), 3.0); - assertSketchEstimate(4); - } - - @Test - public void testUpdateSketchStrings() - { - updateSketch("foo", null, "bar", ""); - assertSketchEstimate(2); - } - - @Test - public void testUpdateSketchListsOfStrings() - { - updateSketch( - Arrays.asList("1", "2"), - Arrays.asList("2", "", "3", "11"), - Arrays.asList("1", null, "3", "12"), - Arrays.asList("1", "3", "13") - ); - - assertSketchEstimate(6); - } - - @Test - public void testUpdateSketchCharArray() - { - updateSketch( - new char[]{1, 2}, - new char[]{2, 3, 11}, - new char[]{1, 2}, - new char[]{1, 3, 13} - ); - - assertSketchEstimate(3); - } - - @Test - public void testUpdateSketchByteArray() - { - updateSketch( - new byte[]{1, 2}, - new byte[]{2, 3, 11}, - new byte[]{1, 2}, - new byte[]{1, 3, 13} - ); - - assertSketchEstimate(3); - } - - @Test - public void testUpdateSketchIntArray() - { - updateSketch( - new int[]{1, 2}, - new int[]{2, 3, 11}, - new int[]{1, 2}, - new int[]{1, 3, 13} - ); - - assertSketchEstimate(3); - } - - @Test - public void testUpdateSketchLongArray() - { - updateSketch( - new long[]{1, 2}, - new long[]{2, 3, 11}, - new long[]{1, 2}, - new long[]{1, 3, 13} - ); - - assertSketchEstimate(3); - } - - private void updateSketch(final Object first, final Object... others) - { - // first != null check mimics how updateSketch is called: it's always guarded by a null check on the outer value. - if (first != null) { - HllSketchBuildAggregator.updateSketch(sketch, first); - } - - for (final Object o : others) { - if (o != null) { - HllSketchBuildAggregator.updateSketch(sketch, o); - } - } - } - - private void assertSketchEstimate(final long estimate) - { - Assert.assertEquals((double) estimate, sketch.getEstimate(), 0.1); - } -} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildUtilTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildUtilTest.java new file mode 100644 index 00000000000..eca5e6f37a4 --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildUtilTest.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.druid.query.aggregation.datasketches.hll; + +import com.google.common.collect.ImmutableMap; +import org.apache.datasketches.hll.HllSketch; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.DimensionDictionarySelector; +import org.apache.druid.segment.IdLookup; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Map; + +/** + * Tests for {@link HllSketchBuildUtil#updateSketch}. + */ +public class HllSketchBuildUtilTest extends InitializedNullHandlingTest +{ + private static final Map DICTIONARY = ImmutableMap.of( + 1, "bar", + 2, "foo" + ); + + private final HllSketch sketch = new HllSketch(HllSketch.DEFAULT_LG_K); + + @Test + public void testUpdateSketchListsOfStringsUTF16LE() + { + updateSketch( + StringEncoding.UTF16LE, + Arrays.asList("1", "2"), + Arrays.asList("2", "", "3", "11"), + Arrays.asList("1", null, "3", "12"), + Arrays.asList("1", "3", "13") + ); + + assertSketchEstimate(6); + } + + @Test + public void testUpdateSketchListsOfStringsUTF8() + { + updateSketch( + StringEncoding.UTF16LE, + Arrays.asList("1", "2"), + Arrays.asList("2", "", "3", "11"), + Arrays.asList("1", null, "3", "12"), + Arrays.asList("1", "3", "13") + ); + + assertSketchEstimate(6); + } + + @Test + public void testUpdateSketchCharArray() + { + updateSketch( + StringEncoding.UTF16LE, + new char[]{1, 2}, + new char[]{2, 3, 11}, + new char[]{1, 2}, + new char[]{1, 3, 13} + ); + + assertSketchEstimate(3); + } + + @Test + public void testUpdateSketchByteArray() + { + updateSketch( + StringEncoding.UTF16LE, + new byte[]{1, 2}, + new byte[]{2, 3, 11}, + new byte[]{1, 2}, + new byte[]{1, 3, 13} + ); + + assertSketchEstimate(3); + } + + @Test + public void testUpdateSketchIntArray() + { + updateSketch( + StringEncoding.UTF16LE, + new int[]{1, 2}, + new int[]{2, 3, 11}, + new int[]{1, 2}, + new int[]{1, 3, 13} + ); + + assertSketchEstimate(3); + } + + @Test + public void testUpdateSketchLongArray() + { + updateSketch( + StringEncoding.UTF16LE, + new long[]{1, 2}, + new long[]{2, 3, 11}, + new long[]{1, 2}, + new long[]{1, 3, 13} + ); + + assertSketchEstimate(3); + } + + @Test + public void testUpdateSketchWithDictionarySelector8to8() + { + final TestDictionarySelector selector = new TestDictionarySelector(DICTIONARY, true); + updateSketchWithDictionarySelector(StringEncoding.UTF8, selector, 0, 1, 2, 1); + assertSketchEstimate(2); + } + + @Test + public void testUpdateSketchWithDictionarySelector8to16() + { + final TestDictionarySelector selector = new TestDictionarySelector(DICTIONARY, true); + updateSketchWithDictionarySelector(StringEncoding.UTF16LE, selector, 0, 1, 2, 1); + assertSketchEstimate(2); + } + + @Test + public void testUpdateSketchWithDictionarySelector16to8() + { + final TestDictionarySelector selector = new TestDictionarySelector(DICTIONARY, false); + updateSketchWithDictionarySelector(StringEncoding.UTF8, selector, 0, 1, 2, 1); + assertSketchEstimate(2); + } + + @Test + public void testUpdateSketchWithDictionarySelector16to16() + { + final TestDictionarySelector selector = new TestDictionarySelector(DICTIONARY, false); + updateSketchWithDictionarySelector(StringEncoding.UTF16LE, selector, 0, 1, 2, 1); + assertSketchEstimate(2); + } + + @Test + public void testUpdateSketchWithDictionarySelectorMixedTo8() + { + final TestDictionarySelector selector1 = new TestDictionarySelector(DICTIONARY, false); + final TestDictionarySelector selector2 = new TestDictionarySelector(DICTIONARY, true); + updateSketchWithDictionarySelector(StringEncoding.UTF8, selector1, 0, 1, 2, 1); + updateSketchWithDictionarySelector(StringEncoding.UTF8, selector2, 0, 1, 2, 1); + assertSketchEstimate(2); // Duplicates are de-duplicated + } + + @Test + public void testUpdateSketchWithDictionarySelectorMixedTo16() + { + final TestDictionarySelector selector1 = new TestDictionarySelector(DICTIONARY, false); + final TestDictionarySelector selector2 = new TestDictionarySelector(DICTIONARY, true); + updateSketchWithDictionarySelector(StringEncoding.UTF16LE, selector1, 0, 1, 2, 1); + updateSketchWithDictionarySelector(StringEncoding.UTF16LE, selector2, 0, 1, 2, 1); + assertSketchEstimate(2); // Duplicates are de-duplicated + } + + @Test + public void testUpdateSketchWithDictionarySelector8ToMixed() + { + final TestDictionarySelector selector = new TestDictionarySelector(DICTIONARY, true); + updateSketchWithDictionarySelector(StringEncoding.UTF8, selector, 0, 1, 2, 1); + updateSketchWithDictionarySelector(StringEncoding.UTF16LE, selector, 0, 1, 2, 1); + assertSketchEstimate(4); // Incompatible hashes + } + + @Test + public void testUpdateSketchWithDictionarySelector16ToMixed() + { + final TestDictionarySelector selector = new TestDictionarySelector(DICTIONARY, false); + updateSketchWithDictionarySelector(StringEncoding.UTF8, selector, 0, 1, 2, 1); + updateSketchWithDictionarySelector(StringEncoding.UTF16LE, selector, 0, 1, 2, 1); + assertSketchEstimate(4); // Incompatible hashes + } + + private void updateSketch(final StringEncoding stringEncoding, final Object first, final Object... others) + { + // first != null check mimics how updateSketch is called: it's always guarded by a null check on the outer value. + if (first != null) { + HllSketchBuildUtil.updateSketch(sketch, stringEncoding, first); + } + + for (final Object o : others) { + if (o != null) { + HllSketchBuildUtil.updateSketch(sketch, stringEncoding, o); + } + } + } + + private void updateSketchWithDictionarySelector( + final StringEncoding stringEncoding, + final DimensionDictionarySelector selector, + final int... ids + ) + { + for (int id : ids) { + HllSketchBuildUtil.updateSketchWithDictionarySelector(sketch, stringEncoding, selector, id); + } + } + + private void assertSketchEstimate(final long estimate) + { + Assert.assertEquals((double) estimate, sketch.getEstimate(), 0.1); + } + + private static class TestDictionarySelector implements DimensionDictionarySelector + { + private final Map dictionary; + private final boolean supportsLookupNameUtf8; + + public TestDictionarySelector(final Map dictionary, final boolean supportsLookupNameUtf8) + { + this.dictionary = dictionary; + this.supportsLookupNameUtf8 = supportsLookupNameUtf8; + } + + @Override + public int getValueCardinality() + { + // Unused by this test + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public String lookupName(int id) + { + return dictionary.get(id); + } + + @Nullable + @Override + public ByteBuffer lookupNameUtf8(int id) + { + if (!supportsLookupNameUtf8) { + throw new UnsupportedOperationException(); + } + + final String s = dictionary.get(id); + + if (s == null) { + return null; + } else { + return ByteBuffer.wrap(StringUtils.toUtf8(s)); + } + } + + @Override + public boolean supportsLookupNameUtf8() + { + return supportsLookupNameUtf8; + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + return true; + } + + @Nullable + @Override + public IdLookup idLookup() + { + return null; + } + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactoryTest.java index ee1bd965e19..101b25b99be 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactoryTest.java @@ -19,18 +19,26 @@ package org.apache.druid.query.aggregation.datasketches.hll; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.datasketches.hll.TgtHllType; +import org.apache.druid.java.util.common.StringEncoding; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException; +import org.apache.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.io.IOException; + public class HllSketchMergeAggregatorFactoryTest { private static final String NAME = "name"; private static final String FIELD_NAME = "fieldName"; private static final int LG_K = 2; private static final String TGT_HLL_TYPE = TgtHllType.HLL_6.name(); + private static final StringEncoding STRING_ENCODING = StringEncoding.UTF16LE; private static final boolean SHOULD_FINALIZE = true; private static final boolean ROUND = true; @@ -40,8 +48,24 @@ public class HllSketchMergeAggregatorFactoryTest @Before public void setUp() { - targetRound = new HllSketchMergeAggregatorFactory(NAME, FIELD_NAME, LG_K, TGT_HLL_TYPE, SHOULD_FINALIZE, ROUND); - targetNoRound = new HllSketchMergeAggregatorFactory(NAME, FIELD_NAME, LG_K, TGT_HLL_TYPE, SHOULD_FINALIZE, !ROUND); + targetRound = new HllSketchMergeAggregatorFactory( + NAME, + FIELD_NAME, + LG_K, + TGT_HLL_TYPE, + STRING_ENCODING, + SHOULD_FINALIZE, + ROUND + ); + targetNoRound = new HllSketchMergeAggregatorFactory( + NAME, + FIELD_NAME, + LG_K, + TGT_HLL_TYPE, + STRING_ENCODING, + SHOULD_FINALIZE, + !ROUND + ); } @Test(expected = AggregatorFactoryNotMergeableException.class) @@ -52,6 +76,7 @@ public class HllSketchMergeAggregatorFactoryTest FIELD_NAME, LG_K, TGT_HLL_TYPE, + STRING_ENCODING, SHOULD_FINALIZE, ROUND ); @@ -66,12 +91,29 @@ public class HllSketchMergeAggregatorFactoryTest FIELD_NAME, LG_K, TGT_HLL_TYPE, + STRING_ENCODING, SHOULD_FINALIZE, ROUND ); targetRound.getMergingFactory(other); } + @Test(expected = AggregatorFactoryNotMergeableException.class) + public void testGetMergingFactoryDifferentStringEncoding() throws Exception + { + HllSketchMergeAggregatorFactory other = new HllSketchMergeAggregatorFactory( + NAME, + FIELD_NAME, + LG_K, + TGT_HLL_TYPE, + StringEncoding.UTF8, + SHOULD_FINALIZE, + ROUND + ); + HllSketchAggregatorFactory result = (HllSketchAggregatorFactory) targetRound.getMergingFactory(other); + Assert.assertEquals(LG_K, result.getLgK()); + } + @Test public void testGetMergingFactoryOtherSmallerLgK() throws Exception { @@ -81,6 +123,7 @@ public class HllSketchMergeAggregatorFactoryTest FIELD_NAME, smallerLgK, TGT_HLL_TYPE, + STRING_ENCODING, SHOULD_FINALIZE, ROUND ); @@ -97,6 +140,7 @@ public class HllSketchMergeAggregatorFactoryTest FIELD_NAME, largerLgK, TGT_HLL_TYPE, + STRING_ENCODING, SHOULD_FINALIZE, ROUND ); @@ -113,6 +157,7 @@ public class HllSketchMergeAggregatorFactoryTest FIELD_NAME, LG_K, smallerTgtHllType, + STRING_ENCODING, SHOULD_FINALIZE, ROUND ); @@ -129,6 +174,7 @@ public class HllSketchMergeAggregatorFactoryTest FIELD_NAME, LG_K, largerTgtHllType, + STRING_ENCODING, SHOULD_FINALIZE, ROUND ); @@ -164,6 +210,80 @@ public class HllSketchMergeAggregatorFactoryTest Assert.assertTrue(result.isRound()); } + @Test + public void testSerde() throws IOException + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper().copy(); + jsonMapper.registerModules(new HllSketchModule().getJacksonModules()); + + final HllSketchMergeAggregatorFactory factory = new HllSketchMergeAggregatorFactory( + "foo", + "bar", + 18, + TgtHllType.HLL_8.name(), + StringEncoding.UTF8, + false, + true + ); + + final String serializedString = jsonMapper.writeValueAsString(factory); + + Assert.assertEquals( + "{\"type\":\"HLLSketchMerge\",\"name\":\"foo\",\"fieldName\":\"bar\",\"lgK\":18,\"tgtHllType\":\"HLL_8\"," + + "\"stringEncoding\":\"utf8\",\"shouldFinalize\":false,\"round\":true}", + serializedString + ); + + final AggregatorFactory factory2 = jsonMapper.readValue( + serializedString, + AggregatorFactory.class + ); + + Assert.assertEquals(factory, factory2); + } + + @Test + public void testSerdeWithDefaults() throws IOException + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper().copy(); + jsonMapper.registerModules(new HllSketchModule().getJacksonModules()); + + final HllSketchMergeAggregatorFactory factory = new HllSketchMergeAggregatorFactory( + "foo", + "bar", + null, + null, + null, + null, + false + ); + + final String serializedString = jsonMapper.writeValueAsString(factory); + + Assert.assertEquals( + "{\"type\":\"HLLSketchMerge\"," + + "\"name\":\"foo\"," + + "\"fieldName\":\"bar\"," + + "\"lgK\":12," + + "\"tgtHllType\":\"HLL_4\"" + + "}", + serializedString + ); + + final AggregatorFactory factory2 = jsonMapper.readValue( + serializedString, + AggregatorFactory.class + ); + + Assert.assertEquals(factory, factory2); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(HllSketchBuildAggregatorFactory.class).usingGetClass().verify(); + } + @Test public void testWithName() throws Exception { diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregatorTest.java index 2c4a31ef716..f84e318fb04 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregatorTest.java @@ -94,6 +94,7 @@ public class HllSketchToEstimatePostAggregatorTest null, null, null, + null, false ) ) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java index 99c93a1cedd..498bb06d9af 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Injector; import org.apache.druid.common.config.NullHandling; import org.apache.druid.guice.DruidInjectorBuilder; +import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.PeriodGranularity; @@ -32,10 +33,12 @@ import org.apache.druid.query.BaseQuery; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.datasketches.SketchQueryContext; import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.hll.HllSketchMergeAggregatorFactory; @@ -61,6 +64,8 @@ import org.apache.druid.query.topn.NumericTopNMetricSpec; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.JoinableFactoryWrapper; @@ -84,11 +89,128 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Properties; +import java.util.stream.Collectors; public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest { private static final boolean ROUND = true; + // For testHllSketchPostAggsGroupBy, testHllSketchPostAggsTimeseries + private static final Object[] EXPECTED_PA_RESULT = + new Object[]{ + "\"AgEHDAMIAgDhUv8P63iABQ==\"", + "\"AgEHDAMIBgALpZ0PjpTfBY5ElQo+C7UE4jA+DKfcYQQ=\"", + "\"AgEHDAMIAQAr8vsG\"", + 2.000000004967054d, + 3.000000004967054d, + 3.000000014901161d, + 2.000000004967054d, + "[2.000000004967054,2.0,2.0001997319422404]", + "[2.000000004967054,2.0,2.000099863468538]", + "\"AgEHDAMIBgC1EYgH1mlHBwsKPwu5SK8MIiUxB7iZVwU=\"", + 2L, + "### HLL SKETCH SUMMARY: \n" + + " Log Config K : 12\n" + + " Hll Target : HLL_4\n" + + " Current Mode : LIST\n" + + " Memory : false\n" + + " LB : 2.0\n" + + " Estimate : 2.000000004967054\n" + + " UB : 2.000099863468538\n" + + " OutOfOrder Flag: false\n" + + " Coupon Count : 2\n", + "### HLL SKETCH SUMMARY: \n" + + " LOG CONFIG K : 12\n" + + " HLL TARGET : HLL_4\n" + + " CURRENT MODE : LIST\n" + + " MEMORY : FALSE\n" + + " LB : 2.0\n" + + " ESTIMATE : 2.000000004967054\n" + + " UB : 2.000099863468538\n" + + " OUTOFORDER FLAG: FALSE\n" + + " COUPON COUNT : 2\n", + 2.0, + 2L + }; + + /** + * Expected virtual columns for {@link #testHllSketchPostAggsTimeseries()}, + * {@link #testHllSketchPostAggsGroupBy()}, {@link #testHllSketchFilteredAggregatorsTimeseries()}, and + * {@link #testHllSketchFilteredAggregatorsGroupBy()}. + */ + private static final List EXPECTED_PA_VIRTUAL_COLUMNS = + ImmutableList.of( + new ExpressionVirtualColumn( + "v0", + "concat(\"dim2\",'hello')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "v1", + "pow(abs((\"m1\" + 100)),2)", + ColumnType.DOUBLE, + TestExprMacroTable.INSTANCE + ) + ); + + /** + * Expected aggregators for {@link #testHllSketchPostAggsTimeseries()} and {@link #testHllSketchPostAggsGroupBy()}. + */ + private static final List EXPECTED_PA_AGGREGATORS = + ImmutableList.of( + new HllSketchBuildAggregatorFactory("a0", "dim2", null, null, null, false, true), + new HllSketchBuildAggregatorFactory("a1", "m1", null, null, null, false, true), + new HllSketchBuildAggregatorFactory("a2", "cnt", null, null, null, false, true), + new HllSketchBuildAggregatorFactory("a3", "v0", null, null, null, false, true), + new HllSketchBuildAggregatorFactory("a4", "v1", null, null, null, false, true), + new HllSketchBuildAggregatorFactory("a5", "dim2", null, null, null, true, true), + new HllSketchBuildAggregatorFactory("a6", "dim2", null, null, StringEncoding.UTF8, true, true) + ); + + /** + * Expected aggregators for {@link #testHllSketchFilteredAggregatorsTimeseries()} and + * {@link #testHllSketchFilteredAggregatorsGroupBy()}. + */ + private static final List EXPECTED_FILTERED_AGGREGATORS = + EXPECTED_PA_AGGREGATORS.stream() + .limit(5) + .map(factory -> new FilteredAggregatorFactory(factory, selector("dim2", "a", null))) + .collect(Collectors.toList()); + + /** + * Expected post-aggregators for {@link #testHllSketchPostAggsTimeseries()} and + * {@link #testHllSketchPostAggsGroupBy()}. + */ + private static final List EXPECTED_PA_POST_AGGREGATORS = + ImmutableList.of( + new HllSketchToEstimatePostAggregator("p1", new FieldAccessPostAggregator("p0", "a0"), false), + new HllSketchToEstimatePostAggregator("p3", new FieldAccessPostAggregator("p2", "a0"), false), + new ExpressionPostAggregator("p4", "(\"p3\" + 1)", null, TestExprMacroTable.INSTANCE), + new HllSketchToEstimatePostAggregator("p6", new FieldAccessPostAggregator("p5", "a3"), false), + new HllSketchToEstimatePostAggregator("p8", new FieldAccessPostAggregator("p7", "a0"), false), + new ExpressionPostAggregator("p9", "abs(\"p8\")", null, TestExprMacroTable.INSTANCE), + new HllSketchToEstimateWithBoundsPostAggregator("p11", new FieldAccessPostAggregator("p10", "a0"), 2), + new HllSketchToEstimateWithBoundsPostAggregator("p13", new FieldAccessPostAggregator("p12", "a0"), 1), + new HllSketchToStringPostAggregator("p15", new FieldAccessPostAggregator("p14", "a0")), + new HllSketchToStringPostAggregator("p17", new FieldAccessPostAggregator("p16", "a0")), + new ExpressionPostAggregator("p18", "upper(\"p17\")", null, TestExprMacroTable.INSTANCE), + new HllSketchToEstimatePostAggregator("p20", new FieldAccessPostAggregator("p19", "a0"), true) + ); + + /** + * Expected post-aggregators for {@link #testHllSketchFilteredAggregatorsTimeseries()} and + * {@link #testHllSketchFilteredAggregatorsGroupBy()}. + */ + private static final List EXPECTED_FILTERED_POST_AGGREGATORS = + ImmutableList.of( + new HllSketchToEstimatePostAggregator("p1", new FieldAccessPostAggregator("p0", "a0"), false), + new HllSketchToEstimatePostAggregator("p3", new FieldAccessPostAggregator("p2", "a1"), false), + new HllSketchToEstimatePostAggregator("p5", new FieldAccessPostAggregator("p4", "a2"), false), + new HllSketchToEstimatePostAggregator("p7", new FieldAccessPostAggregator("p6", "a3"), false), + new HllSketchToEstimatePostAggregator("p9", new FieldAccessPostAggregator("p8", "a4"), false) + ); + private static final ExprMacroTable MACRO_TABLE = new ExprMacroTable( ImmutableList.of( new HllPostAggExprMacros.HLLSketchEstimateExprMacro() @@ -133,6 +255,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest "dim1", null, null, + null, false, ROUND ), @@ -141,6 +264,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest "dim3", null, null, + null, false, false ) @@ -217,44 +341,16 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest .aggregators( ImmutableList.of( new LongSumAggregatorFactory("a0", "cnt"), - new HllSketchBuildAggregatorFactory( - "a1", - "dim2", - null, - null, - null, - ROUND - ), + new HllSketchBuildAggregatorFactory("a1", "dim2", null, null, null, null, ROUND), new FilteredAggregatorFactory( - new HllSketchBuildAggregatorFactory( - "a2", - "dim2", - null, - null, - null, - ROUND - ), + new HllSketchBuildAggregatorFactory("a2", "dim2", null, null, null, null, ROUND), BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null)) ), - new HllSketchBuildAggregatorFactory( - "a3", - "v0", - null, - null, - null, - ROUND - ), - new HllSketchBuildAggregatorFactory( - "a4", - "v1", - null, - null, - null, - ROUND - ), - new HllSketchMergeAggregatorFactory("a5", "hllsketch_dim1", 21, "HLL_8", null, ROUND), - new HllSketchMergeAggregatorFactory("a6", "hllsketch_dim1", null, null, null, ROUND), - new HllSketchMergeAggregatorFactory("a7", "hllsketch_dim1", 21, "HLL_4", null, ROUND) + new HllSketchBuildAggregatorFactory("a3", "v0", null, null, null, null, ROUND), + new HllSketchBuildAggregatorFactory("a4", "v1", null, null, null, null, ROUND), + new HllSketchMergeAggregatorFactory("a5", "hllsketch_dim1", 21, "HLL_8", null, null, ROUND), + new HllSketchMergeAggregatorFactory("a6", "hllsketch_dim1", null, null, null, null, ROUND), + new HllSketchMergeAggregatorFactory("a7", "hllsketch_dim1", 21, "HLL_4", null, null, ROUND) ) ) .context(QUERY_CONTEXT_DEFAULT) @@ -303,6 +399,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest null, null, null, + null, ROUND ) ) @@ -380,7 +477,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest .setGranularity(Granularities.ALL) .setAggregatorSpecs( aggregators( - new HllSketchBuildAggregatorFactory("a0", "m1", null, null, null, true) + new HllSketchBuildAggregatorFactory("a0", "m1", null, null, null, true, true) ) ) .setHavingSpec(having(selector("a0", "2", null))) @@ -399,33 +496,103 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest } @Test - public void testHllSketchPostAggs() + public void testHllSketchFilteredAggregatorsGroupBy() { - final String sketchSummary = "### HLL SKETCH SUMMARY: \n" - + " Log Config K : 12\n" - + " Hll Target : HLL_4\n" - + " Current Mode : LIST\n" - + " Memory : false\n" - + " LB : 2.0\n" - + " Estimate : 2.000000004967054\n" - + " UB : 2.000099863468538\n" - + " OutOfOrder Flag: false\n" - + " Coupon Count : 2\n"; + testQuery( + "SELECT\n" + + " DS_HLL(dim2) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " DS_HLL(m1) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " DS_HLL(cnt) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " DS_HLL(CONCAT(dim2, 'hello')) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " DS_HLL(POWER(ABS(m1 + 100), 2)) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(dim2) FILTER(WHERE MV_CONTAINS(dim2, 'a'))),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(m1) FILTER(WHERE MV_CONTAINS(dim2, 'a'))),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(cnt) FILTER(WHERE MV_CONTAINS(dim2, 'a'))),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(CONCAT(dim2, 'hello')) FILTER(WHERE MV_CONTAINS(dim2, 'a'))),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(POWER(ABS(m1 + 100), 2)) FILTER(WHERE MV_CONTAINS(dim2, 'a')))\n" + + "FROM druid.foo\n" + + "GROUP BY cnt", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(new MultipleIntervalSegmentSpec(Collections.singletonList(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setVirtualColumns(VirtualColumns.create(EXPECTED_PA_VIRTUAL_COLUMNS)) + .setDimensions(new DefaultDimensionSpec("cnt", "d0", ColumnType.LONG)) + .setAggregatorSpecs(EXPECTED_FILTERED_AGGREGATORS) + .setPostAggregatorSpecs(EXPECTED_FILTERED_POST_AGGREGATORS) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{ + "\"AgEHDAMIAQDhUv8P\"", + "\"AgEHDAMIAgALpZ0PPgu1BA==\"", + "\"AgEHDAMIAQAr8vsG\"", + "\"AgEHDAMIAQCba0kG\"", + "\"AgEHDAMIAgC1EYgHuUivDA==\"", + 1.0, + 2.000000004967054, + 1.0, + 1.0, + 2.000000004967054 + } + ) + ); + } - final String otherSketchSummary = "### HLL SKETCH SUMMARY: \n" - + " LOG CONFIG K : 12\n" - + " HLL TARGET : HLL_4\n" - + " CURRENT MODE : LIST\n" - + " MEMORY : FALSE\n" - + " LB : 2.0\n" - + " ESTIMATE : 2.000000004967054\n" - + " UB : 2.000099863468538\n" - + " OUTOFORDER FLAG: FALSE\n" - + " COUPON COUNT : 2\n"; + @Test + public void testHllSketchFilteredAggregatorsTimeseries() + { + testQuery( + "SELECT\n" + + " DS_HLL(dim2) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " DS_HLL(m1) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " DS_HLL(cnt) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " DS_HLL(CONCAT(dim2, 'hello')) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " DS_HLL(POWER(ABS(m1 + 100), 2)) FILTER(WHERE MV_CONTAINS(dim2, 'a')),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(dim2) FILTER(WHERE MV_CONTAINS(dim2, 'a'))),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(m1) FILTER(WHERE MV_CONTAINS(dim2, 'a'))),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(cnt) FILTER(WHERE MV_CONTAINS(dim2, 'a'))),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(CONCAT(dim2, 'hello')) FILTER(WHERE MV_CONTAINS(dim2, 'a'))),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(POWER(ABS(m1 + 100), 2)) FILTER(WHERE MV_CONTAINS(dim2, 'a')))\n" + + "FROM druid.foo", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .virtualColumns(VirtualColumns.create(EXPECTED_PA_VIRTUAL_COLUMNS)) + .aggregators(EXPECTED_FILTERED_AGGREGATORS) + .postAggregators(EXPECTED_FILTERED_POST_AGGREGATORS) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{ + "\"AgEHDAMIAQDhUv8P\"", + "\"AgEHDAMIAgALpZ0PPgu1BA==\"", + "\"AgEHDAMIAQAr8vsG\"", + "\"AgEHDAMIAQCba0kG\"", + "\"AgEHDAMIAgC1EYgHuUivDA==\"", + 1.0, + 2.000000004967054, + 1.0, + 1.0, + 2.000000004967054 + } + ) + ); + } + + @Test + public void testHllSketchPostAggsGroupBy() + { testQuery( "SELECT\n" + " DS_HLL(dim2),\n" + " DS_HLL(m1),\n" + + " DS_HLL(cnt),\n" + " HLL_SKETCH_ESTIMATE(DS_HLL(dim2)),\n" + " HLL_SKETCH_ESTIMATE(DS_HLL(dim2)) + 1,\n" + " HLL_SKETCH_ESTIMATE(DS_HLL(CONCAT(dim2, 'hello'))),\n" @@ -436,119 +603,59 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest + " APPROX_COUNT_DISTINCT_DS_HLL(dim2),\n" + " HLL_SKETCH_TO_STRING(DS_HLL(dim2)),\n" + " UPPER(HLL_SKETCH_TO_STRING(DS_HLL(dim2))),\n" - + " HLL_SKETCH_ESTIMATE(DS_HLL(dim2), true)\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(dim2), true)\n," + + " APPROX_COUNT_DISTINCT_DS_HLL_UTF8(dim2)\n" + + "FROM druid.foo\n" + + "GROUP BY cnt", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(new MultipleIntervalSegmentSpec(Collections.singletonList(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setVirtualColumns(VirtualColumns.create(EXPECTED_PA_VIRTUAL_COLUMNS)) + .setDimensions(new DefaultDimensionSpec("cnt", "d0", ColumnType.LONG)) + .setAggregatorSpecs(EXPECTED_PA_AGGREGATORS) + .setPostAggregatorSpecs(EXPECTED_PA_POST_AGGREGATORS) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of(EXPECTED_PA_RESULT) + ); + } + + @Test + public void testHllSketchPostAggsTimeseries() + { + testQuery( + "SELECT\n" + + " DS_HLL(dim2),\n" + + " DS_HLL(m1),\n" + + " DS_HLL(cnt),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(dim2)),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(dim2)) + 1,\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(CONCAT(dim2, 'hello'))),\n" + + " ABS(HLL_SKETCH_ESTIMATE(DS_HLL(dim2))),\n" + + " HLL_SKETCH_ESTIMATE_WITH_ERROR_BOUNDS(DS_HLL(dim2), 2),\n" + + " HLL_SKETCH_ESTIMATE_WITH_ERROR_BOUNDS(DS_HLL(dim2)),\n" + + " DS_HLL(POWER(ABS(m1 + 100), 2)),\n" + + " APPROX_COUNT_DISTINCT_DS_HLL(dim2),\n" + + " HLL_SKETCH_TO_STRING(DS_HLL(dim2)),\n" + + " UPPER(HLL_SKETCH_TO_STRING(DS_HLL(dim2))),\n" + + " HLL_SKETCH_ESTIMATE(DS_HLL(dim2), true),\n" + + " APPROX_COUNT_DISTINCT_DS_HLL_UTF8(dim2)\n" + "FROM druid.foo", ImmutableList.of( Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) - .virtualColumns( - new ExpressionVirtualColumn( - "v0", - "concat(\"dim2\",'hello')", - ColumnType.STRING, - TestExprMacroTable.INSTANCE - ), - new ExpressionVirtualColumn( - "v1", - "pow(abs((\"m1\" + 100)),2)", - ColumnType.DOUBLE, - TestExprMacroTable.INSTANCE - ) - ) - .aggregators( - ImmutableList.of( - new HllSketchBuildAggregatorFactory( - "a0", - "dim2", - null, - null, - false, - true - ), - new HllSketchBuildAggregatorFactory( - "a1", - "m1", - null, - null, - false, - true - ), - new HllSketchBuildAggregatorFactory( - "a2", - "v0", - null, - null, - false, - true - ), - new HllSketchBuildAggregatorFactory( - "a3", - "v1", - null, - null, - false, - true - ), - new HllSketchBuildAggregatorFactory( - "a4", - "dim2", - null, - null, - null, - true - ) - ) - ) - .postAggregators( - ImmutableList.of( - new HllSketchToEstimatePostAggregator("p1", new FieldAccessPostAggregator("p0", "a0"), false), - new HllSketchToEstimatePostAggregator("p3", new FieldAccessPostAggregator("p2", "a0"), false), - new ExpressionPostAggregator("p4", "(\"p3\" + 1)", null, TestExprMacroTable.INSTANCE), - new HllSketchToEstimatePostAggregator("p6", new FieldAccessPostAggregator("p5", "a2"), false), - new HllSketchToEstimatePostAggregator( - "p8", - new FieldAccessPostAggregator("p7", "a0"), - false - ), - new ExpressionPostAggregator("p9", "abs(\"p8\")", null, TestExprMacroTable.INSTANCE), - new HllSketchToEstimateWithBoundsPostAggregator( - "p11", - new FieldAccessPostAggregator("p10", "a0"), - 2 - ), - new HllSketchToEstimateWithBoundsPostAggregator( - "p13", - new FieldAccessPostAggregator("p12", "a0"), - 1 - ), - new HllSketchToStringPostAggregator("p15", new FieldAccessPostAggregator("p14", "a0")), - new HllSketchToStringPostAggregator("p17", new FieldAccessPostAggregator("p16", "a0")), - new ExpressionPostAggregator("p18", "upper(\"p17\")", null, TestExprMacroTable.INSTANCE), - new HllSketchToEstimatePostAggregator("p20", new FieldAccessPostAggregator("p19", "a0"), true) - ) - ) + .virtualColumns(VirtualColumns.create(EXPECTED_PA_VIRTUAL_COLUMNS)) + .aggregators(EXPECTED_PA_AGGREGATORS) + .postAggregators(EXPECTED_PA_POST_AGGREGATORS) .context(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of( - new Object[]{ - "\"AgEHDAMIAgDhUv8P63iABQ==\"", - "\"AgEHDAMIBgALpZ0PjpTfBY5ElQo+C7UE4jA+DKfcYQQ=\"", - 2.000000004967054d, - 3.000000004967054d, - 3.000000014901161d, - 2.000000004967054d, - "[2.000000004967054,2.0,2.0001997319422404]", - "[2.000000004967054,2.0,2.000099863468538]", - "\"AgEHDAMIBgC1EYgH1mlHBwsKPwu5SK8MIiUxB7iZVwU=\"", - 2L, - sketchSummary, - otherSketchSummary, - 2.0 - } - ) + ImmutableList.of(EXPECTED_PA_RESULT) ); } @@ -620,46 +727,11 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest ) .aggregators( ImmutableList.of( - new HllSketchBuildAggregatorFactory( - "a0", - "dim2", - null, - null, - null, - true - ), - new HllSketchBuildAggregatorFactory( - "a1", - "m1", - null, - null, - null, - true - ), - new HllSketchBuildAggregatorFactory( - "a2", - "v0", - null, - null, - null, - true - ), - new HllSketchBuildAggregatorFactory( - "a3", - "v1", - null, - null, - null, - true - ), - new HllSketchBuildAggregatorFactory( - "a4", - "dim2", - null, - null, - null, - true - ) + new HllSketchBuildAggregatorFactory("a0", "dim2", null, null, null, true, true), + new HllSketchBuildAggregatorFactory("a1", "m1", null, null, null, true, true), + new HllSketchBuildAggregatorFactory("a2", "v0", null, null, null, true, true), + new HllSketchBuildAggregatorFactory("a3", "v1", null, null, null, true, true), + new HllSketchBuildAggregatorFactory("a4", "dim2", null, null, null, true, true) ) ) .postAggregators( @@ -743,6 +815,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest "dim2", null, null, + null, false, true ) @@ -789,6 +862,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest null, null, null, + null, true ), new HllSketchBuildAggregatorFactory( @@ -796,6 +870,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest "dim2", null, null, + null, false, true ) @@ -833,6 +908,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest null, null, null, + null, true ), selector("dim1", "nonexistent", null) @@ -843,6 +919,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest "v0", null, null, + null, false, true ), @@ -884,25 +961,11 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( - new HllSketchBuildAggregatorFactory( - "a0", - "v0", - null, - null, - null, - true - ), + new HllSketchBuildAggregatorFactory("a0", "v0", null, null, null, null, true), selector("dim1", "nonexistent", null) ), new FilteredAggregatorFactory( - new HllSketchBuildAggregatorFactory( - "a1", - "v0", - null, - null, - null, - true - ), + new HllSketchBuildAggregatorFactory("a1", "v0", null, null, null, null, true), selector("dim1", "nonexistent", null) ) ) diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index fccaa83f3e3..3c40affa783 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -172,7 +172,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest // FloatSumAggregator combine method takes in two Float but return Double new FloatSumAggregatorFactory("sum_added", "added"), new SketchMergeAggregatorFactory("thetaSketch", "user", 16384, true, false, null), - new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", 12, TgtHllType.HLL_4.name(), false, false), + new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", 12, TgtHllType.HLL_4.name(), null, false, false), new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", 128, 1000000000L, null) }, false @@ -266,7 +266,15 @@ public class ITAutoCompactionTest extends AbstractIndexerTest new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added"), new SketchMergeAggregatorFactory("thetaSketch", "user", 16384, true, false, null), - new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", 12, TgtHllType.HLL_4.name(), false, false), + new HllSketchBuildAggregatorFactory( + "HLLSketchBuild", + "user", + 12, + TgtHllType.HLL_4.name(), + null, + false, + false + ), new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", 128, 1000000000L, null) }, false diff --git a/processing/src/main/java/org/apache/druid/java/util/common/StringEncoding.java b/processing/src/main/java/org/apache/druid/java/util/common/StringEncoding.java new file mode 100644 index 00000000000..c5775d10b27 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/common/StringEncoding.java @@ -0,0 +1,65 @@ +/* + * 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.druid.java.util.common; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.primitives.SignedBytes; + +/** + * An enum that provides a way for users to specify what encoding should be used when hashing strings. + * + * The main reason for this setting's existence is getting the best performance possible. When operating on memory + * mapped segments -- which store strings as UTF-8 -- it is fastest to use "UTF8". When operating on the result of + * expressions, or on an in-heap IncrementalIndex -- which use Java strings -- it is fastest to use "UTF16LE". + * + * This decision cannot be made locally, because different encodings do not generate equivalent hashes, and therefore + * they are not mergeable. The decision must be made globally by the end user or by the SQL planner, and should be + * based on where most input strings are expected to come from. + * + * Currently, UTF8 and UTF16LE are the only two options, because there are no situations where other options would be + * higher-performing. + */ +public enum StringEncoding implements Cacheable +{ + // Do not change order; the ordinal is used by cache keys. Add new ones at the end. + + UTF8, + UTF16LE /* Treat the result of str.toCharArray() as a bag of bytes in little-endian order */; + + @JsonCreator + public static StringEncoding fromString(final String name) + { + return valueOf(StringUtils.toUpperCase(name)); + } + + @Override + public byte[] getCacheKey() + { + return new byte[]{SignedBytes.checkedCast(ordinal())}; + } + + @JsonValue + @Override + public String toString() + { + return StringUtils.toLowerCase(this.name()); + } +} diff --git a/processing/src/main/java/org/apache/druid/java/util/common/StringEncodingDefaultUTF16LEJsonIncludeFilter.java b/processing/src/main/java/org/apache/druid/java/util/common/StringEncodingDefaultUTF16LEJsonIncludeFilter.java new file mode 100644 index 00000000000..332df030856 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/common/StringEncodingDefaultUTF16LEJsonIncludeFilter.java @@ -0,0 +1,39 @@ +/* + * 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.druid.java.util.common; + +import com.fasterxml.jackson.annotation.JsonInclude; + +/** + * {@link JsonInclude} filter for {@link StringEncoding} that ignores UTF16LE, which is the typical default + * for aggregators. + * + * This API works by "creative" use of equals. It requires warnings to be suppressed and also requires spotbugs + * exclusions (see spotbugs-exclude.xml). + */ +@SuppressWarnings({"EqualsAndHashcode", "EqualsHashCode", "EqualsWhichDoesntCheckParameterClass"}) +public class StringEncodingDefaultUTF16LEJsonIncludeFilter // lgtm [java/inconsistent-equals-and-hashcode] +{ + @Override + public boolean equals(Object obj) + { + return obj == StringEncoding.UTF16LE; + } +} diff --git a/processing/src/test/java/org/apache/druid/java/util/common/StringEncodingDefaultUTF16LEJsonIncludeFilterTest.java b/processing/src/test/java/org/apache/druid/java/util/common/StringEncodingDefaultUTF16LEJsonIncludeFilterTest.java new file mode 100644 index 00000000000..4b93291d779 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/java/util/common/StringEncodingDefaultUTF16LEJsonIncludeFilterTest.java @@ -0,0 +1,37 @@ +/* + * 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.druid.java.util.common; + +import org.junit.Assert; +import org.junit.Test; + +public class StringEncodingDefaultUTF16LEJsonIncludeFilterTest +{ + private final StringEncodingDefaultUTF16LEJsonIncludeFilter filter = + new StringEncodingDefaultUTF16LEJsonIncludeFilter(); + + @Test + @SuppressWarnings({"SimplifiableAssertion", "EqualsBetweenInconvertibleTypes"}) + public void testFilter() + { + Assert.assertTrue(filter.equals(StringEncoding.UTF16LE)); + Assert.assertFalse(filter.equals(StringEncoding.UTF8)); + } +} diff --git a/processing/src/test/java/org/apache/druid/java/util/common/StringEncodingTest.java b/processing/src/test/java/org/apache/druid/java/util/common/StringEncodingTest.java new file mode 100644 index 00000000000..b549e63c872 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/java/util/common/StringEncodingTest.java @@ -0,0 +1,57 @@ +/* + * 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.druid.java.util.common; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; + +public class StringEncodingTest +{ + @Test + public void testSerde() throws IOException + { + final ObjectMapper mapper = new ObjectMapper(); + + Assert.assertEquals( + StringEncoding.UTF16LE, + mapper.readValue(mapper.writeValueAsString(StringEncoding.UTF16LE), StringEncoding.class) + ); + + Assert.assertEquals( + StringEncoding.UTF8, + mapper.readValue(mapper.writeValueAsString(StringEncoding.UTF8), StringEncoding.class) + ); + } + + @Test + public void testGetCacheKey() + { + Assert.assertFalse( + Arrays.equals( + StringEncoding.UTF8.getCacheKey(), + StringEncoding.UTF16LE.getCacheKey() + ) + ); + } +} From 048dbcee8846d51a38b228c8aaac88851278277e Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sat, 1 Jul 2023 13:14:19 -0700 Subject: [PATCH 55/74] MSQ: Improve InsertTimeOutOfBounds error message. (#14511) Nicer and actionable error message for `InsertTimeOutOfBounds` fault --- .../apache/druid/msq/exec/ControllerImpl.java | 2 +- .../error/InsertTimeOutOfBoundsFault.java | 39 +++++++++++++++++-- .../apache/druid/msq/exec/MSQFaultsTest.java | 7 +++- .../msq/indexing/error/MSQFaultSerdeTest.java | 6 ++- 4 files changed, 47 insertions(+), 7 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 72657ed0e28..eebc95077b3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1007,7 +1007,7 @@ public class ControllerImpl implements Controller // Validate interval against the replaceTimeChunks set of intervals. if (destination.getReplaceTimeChunks().stream().noneMatch(chunk -> chunk.contains(interval))) { - throw new MSQException(new InsertTimeOutOfBoundsFault(interval)); + throw new MSQException(new InsertTimeOutOfBoundsFault(interval, destination.getReplaceTimeChunks())); } final List> ranges = bucketEntry.getValue(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeOutOfBoundsFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeOutOfBoundsFault.java index c15a3fb6d17..3a204cb79cf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeOutOfBoundsFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertTimeOutOfBoundsFault.java @@ -22,6 +22,7 @@ package org.apache.druid.msq.indexing.error; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; +import java.util.List; import java.util.Objects; public class InsertTimeOutOfBoundsFault extends BaseMSQFault @@ -29,11 +30,23 @@ public class InsertTimeOutOfBoundsFault extends BaseMSQFault static final String CODE = "InsertTimeOutOfBounds"; private final Interval interval; + private final List intervalBounds; - public InsertTimeOutOfBoundsFault(@JsonProperty("interval") Interval interval) + public InsertTimeOutOfBoundsFault( + @JsonProperty("interval") Interval interval, + @JsonProperty("intervalBounds") List intervalBounds + ) { - super(CODE, "Query generated time chunk [%s] out of bounds specified by replaceExistingTimeChunks", interval); + super( + CODE, + "Inserted data contains time chunk[%s] outside of bounds specified by OVERWRITE WHERE[%s]. " + + "If you want to include this data, expand your OVERWRITE WHERE. " + + "If you do not want to include this data, use SELECT ... WHERE to filter it from your inserted data.", + interval, + intervalBounds + ); this.interval = interval; + this.intervalBounds = intervalBounds; } @JsonProperty @@ -42,6 +55,12 @@ public class InsertTimeOutOfBoundsFault extends BaseMSQFault return interval; } + @JsonProperty + public List getIntervalBounds() + { + return intervalBounds; + } + @Override public boolean equals(Object o) { @@ -55,12 +74,24 @@ public class InsertTimeOutOfBoundsFault extends BaseMSQFault return false; } InsertTimeOutOfBoundsFault that = (InsertTimeOutOfBoundsFault) o; - return Objects.equals(interval, that.interval); + return Objects.equals(interval, that.interval) && Objects.equals( + intervalBounds, + that.intervalBounds + ); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), interval); + return Objects.hash(super.hashCode(), interval, intervalBounds); + } + + @Override + public String toString() + { + return "InsertTimeOutOfBoundsFault{" + + "interval=" + interval + + ", intervalBounds=" + intervalBounds + + '}'; } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java index cfcc1073331..b3bf3c408a2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java @@ -153,7 +153,12 @@ public class MSQFaultsTest extends MSQTestBase "replace into foo1 overwrite where __time >= TIMESTAMP '2002-01-02 00:00:00' and __time < TIMESTAMP '2002-01-03 00:00:00' select __time, dim1 , count(*) as cnt from foo where dim1 is not null group by 1, 2 PARTITIONED by day clustered by dim1") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) - .setExpectedMSQFault(new InsertTimeOutOfBoundsFault(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z"))) + .setExpectedMSQFault( + new InsertTimeOutOfBoundsFault( + Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z"), + Collections.singletonList(Intervals.of("2002-01-02/2002-01-03")) + ) + ) .verifyResults(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index bcd0de1ef68..9e0dbd8accf 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -32,6 +32,7 @@ import org.junit.Test; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; public class MSQFaultSerdeTest { @@ -63,7 +64,10 @@ public class MSQFaultSerdeTest assertFaultSerde(new InsertCannotBeEmptyFault("the datasource")); assertFaultSerde(InsertLockPreemptedFault.INSTANCE); assertFaultSerde(InsertTimeNullFault.INSTANCE); - assertFaultSerde(new InsertTimeOutOfBoundsFault(Intervals.ETERNITY)); + assertFaultSerde(new InsertTimeOutOfBoundsFault( + Intervals.of("2001/2002"), + Collections.singletonList(Intervals.of("2000/2001")) + )); assertFaultSerde(new InvalidNullByteFault("the source", 1, "the column", "the value", 2)); assertFaultSerde(new NotEnoughMemoryFault(1000, 1000, 900, 1, 2)); assertFaultSerde(QueryNotSupportedFault.INSTANCE); From 58f3faf2996051a037555d665da6a8781215e037 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sun, 2 Jul 2023 07:22:52 -0700 Subject: [PATCH 56/74] SortMergeJoinFrameProcessor: Fix two bugs with buffering. (#14196) 1) Fix a problem where the fault wasn't reported when the left-hand side had too many buffered frames. (Instead, frames continued to be buffered, eventually running the server out of memory.) 2) Always update the mark when rewinding isn't necessary. It fixes a problem where frames would be needlessly buffered when there isn't a key match across the two sides. 3) Memory reserved for building the trackers now change based on the heap sized --- .../org/apache/druid/msq/exec/Limits.java | 6 - .../msq/exec/WorkerMemoryParameters.java | 51 +-- .../error/TooManyRowsWithSameKeyFault.java | 8 +- .../msq/querykit/BroadcastJoinHelper.java | 3 +- .../common/SortMergeJoinFrameProcessor.java | 318 ++++++++++++------ .../SortMergeJoinFrameProcessorFactory.java | 3 +- .../msq/exec/WorkerMemoryParametersTest.java | 50 ++- .../SortMergeJoinFrameProcessorTest.java | 287 +++++++++++++++- 8 files changed, 550 insertions(+), 176 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java index c9e598c6188..9069794222f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java @@ -69,12 +69,6 @@ public class Limits */ public static final int MAX_KERNEL_MANIPULATION_QUEUE_SIZE = 100_000; - /** - * Maximum number of bytes buffered for each side of a - * {@link org.apache.druid.msq.querykit.common.SortMergeJoinFrameProcessor}, not counting the most recent frame read. - */ - public static final int MAX_BUFFERED_BYTES_FOR_SORT_MERGE_JOIN = 10_000_000; - /** * Maximum relaunches across all workers. */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java index f64e4dbd0ef..4bddb949f07 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java @@ -137,29 +137,33 @@ public class WorkerMemoryParameters * we use a value somewhat lower than 0.5. */ static final double BROADCAST_JOIN_MEMORY_FRACTION = 0.3; + + /** + * Fraction of free memory per bundle that can be used by + * {@link org.apache.druid.msq.querykit.common.SortMergeJoinFrameProcessor} to buffer frames in its trackers. + */ + static final double SORT_MERGE_JOIN_MEMORY_FRACTION = 0.9; + /** * In case {@link NotEnoughMemoryFault} is thrown, a fixed estimation overhead is added when estimating total memory required for the process. */ private static final long BUFFER_BYTES_FOR_ESTIMATION = 1000; + private final long processorBundleMemory; private final int superSorterMaxActiveProcessors; private final int superSorterMaxChannelsPerProcessor; - private final long appenderatorMemory; - private final long broadcastJoinMemory; private final int partitionStatisticsMaxRetainedBytes; WorkerMemoryParameters( + final long processorBundleMemory, final int superSorterMaxActiveProcessors, final int superSorterMaxChannelsPerProcessor, - final long appenderatorMemory, - final long broadcastJoinMemory, final int partitionStatisticsMaxRetainedBytes ) { + this.processorBundleMemory = processorBundleMemory; this.superSorterMaxActiveProcessors = superSorterMaxActiveProcessors; this.superSorterMaxChannelsPerProcessor = superSorterMaxChannelsPerProcessor; - this.appenderatorMemory = appenderatorMemory; - this.broadcastJoinMemory = broadcastJoinMemory; this.partitionStatisticsMaxRetainedBytes = partitionStatisticsMaxRetainedBytes; } @@ -344,10 +348,9 @@ public class WorkerMemoryParameters } return new WorkerMemoryParameters( + bundleMemoryForProcessing, superSorterMaxActiveProcessors, superSorterMaxChannelsPerProcessor, - (long) (bundleMemoryForProcessing * APPENDERATOR_MEMORY_FRACTION), - (long) (bundleMemoryForProcessing * BROADCAST_JOIN_MEMORY_FRACTION), Ints.checkedCast(workerMemory) // 100% of worker memory is devoted to partition statistics ); } @@ -365,13 +368,13 @@ public class WorkerMemoryParameters public long getAppenderatorMaxBytesInMemory() { // Half for indexing, half for merging. - return Math.max(1, appenderatorMemory / 2); + return Math.max(1, getAppenderatorMemory() / 2); } public int getAppenderatorMaxColumnsToMerge() { // Half for indexing, half for merging. - return Ints.checkedCast(Math.max(2, appenderatorMemory / 2 / APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN)); + return Ints.checkedCast(Math.max(2, getAppenderatorMemory() / 2 / APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN)); } public int getStandardFrameSize() @@ -386,7 +389,12 @@ public class WorkerMemoryParameters public long getBroadcastJoinMemory() { - return broadcastJoinMemory; + return (long) (processorBundleMemory * BROADCAST_JOIN_MEMORY_FRACTION); + } + + public long getSortMergeJoinMemory() + { + return (long) (processorBundleMemory * SORT_MERGE_JOIN_MEMORY_FRACTION); } public int getPartitionStatisticsMaxRetainedBytes() @@ -394,6 +402,14 @@ public class WorkerMemoryParameters return partitionStatisticsMaxRetainedBytes; } + /** + * Amount of memory to devote to {@link org.apache.druid.segment.realtime.appenderator.Appenderator}. + */ + private long getAppenderatorMemory() + { + return (long) (processorBundleMemory * APPENDERATOR_MEMORY_FRACTION); + } + @Override public boolean equals(Object o) { @@ -404,10 +420,9 @@ public class WorkerMemoryParameters return false; } WorkerMemoryParameters that = (WorkerMemoryParameters) o; - return superSorterMaxActiveProcessors == that.superSorterMaxActiveProcessors + return processorBundleMemory == that.processorBundleMemory + && superSorterMaxActiveProcessors == that.superSorterMaxActiveProcessors && superSorterMaxChannelsPerProcessor == that.superSorterMaxChannelsPerProcessor - && appenderatorMemory == that.appenderatorMemory - && broadcastJoinMemory == that.broadcastJoinMemory && partitionStatisticsMaxRetainedBytes == that.partitionStatisticsMaxRetainedBytes; } @@ -415,10 +430,9 @@ public class WorkerMemoryParameters public int hashCode() { return Objects.hash( + processorBundleMemory, superSorterMaxActiveProcessors, superSorterMaxChannelsPerProcessor, - appenderatorMemory, - broadcastJoinMemory, partitionStatisticsMaxRetainedBytes ); } @@ -427,10 +441,9 @@ public class WorkerMemoryParameters public String toString() { return "WorkerMemoryParameters{" + - "superSorterMaxActiveProcessors=" + superSorterMaxActiveProcessors + + "processorBundleMemory=" + processorBundleMemory + + ", superSorterMaxActiveProcessors=" + superSorterMaxActiveProcessors + ", superSorterMaxChannelsPerProcessor=" + superSorterMaxChannelsPerProcessor + - ", appenderatorMemory=" + appenderatorMemory + - ", broadcastJoinMemory=" + broadcastJoinMemory + ", partitionStatisticsMaxRetainedBytes=" + partitionStatisticsMaxRetainedBytes + '}'; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyRowsWithSameKeyFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyRowsWithSameKeyFault.java index 21fa363af89..60d355579b6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyRowsWithSameKeyFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyRowsWithSameKeyFault.java @@ -44,10 +44,12 @@ public class TooManyRowsWithSameKeyFault extends BaseMSQFault { super( CODE, - "Too many rows with the same key during sort-merge join (bytes buffered = %,d; limit = %,d). Key: %s", + "Too many rows with the same key[%s] during sort-merge join (bytes buffered[%,d], limit[%,d]). " + + "Try increasing heap memory available to workers, " + + "or adjusting your query to process fewer rows with this key.", + key, numBytes, - maxBytes, - key + maxBytes ); this.key = key; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java index 36dc52c5cee..d9e7bc6deec 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java @@ -27,6 +27,7 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.processor.FrameProcessors; import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.query.DataSource; @@ -58,7 +59,7 @@ public class BroadcastJoinHelper * @param channels list of input channels * @param channelReaders list of input channel readers; corresponds one-to-one with "channels" * @param memoryReservedForBroadcastJoin total bytes of frames we are permitted to use; derived from - * {@link org.apache.druid.msq.exec.WorkerMemoryParameters#broadcastJoinMemory} + * {@link WorkerMemoryParameters#getBroadcastJoinMemory()} */ public BroadcastJoinHelper( final Int2IntMap inputNumberToProcessorChannelMap, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java index 2c454e1d45c..fdc80560f29 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java @@ -41,7 +41,6 @@ import org.apache.druid.frame.segment.FrameCursor; import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.msq.exec.Limits; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.TooManyRowsWithSameKeyFault; import org.apache.druid.msq.input.ReadableInput; @@ -122,6 +121,7 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor private final String rightPrefix; private final JoinType joinType; private final JoinColumnSelectorFactory joinColumnSelectorFactory = new JoinColumnSelectorFactory(); + private final long maxBufferedBytes; private FrameWriter frameWriter = null; // Used by runIncrementally to defer certain logic to the next run. @@ -137,7 +137,8 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor FrameWriterFactory frameWriterFactory, String rightPrefix, List> keyColumns, - JoinType joinType + JoinType joinType, + long maxBufferedBytes ) { this.inputChannels = ImmutableList.of(left.getChannel(), right.getChannel()); @@ -146,9 +147,10 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor this.rightPrefix = rightPrefix; this.joinType = joinType; this.trackers = ImmutableList.of( - new Tracker(left, keyColumns.get(LEFT)), - new Tracker(right, keyColumns.get(RIGHT)) + new Tracker(left, keyColumns.get(LEFT), maxBufferedBytes), + new Tracker(right, keyColumns.get(RIGHT), maxBufferedBytes) ); + this.maxBufferedBytes = maxBufferedBytes; } @Override @@ -166,10 +168,10 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor @Override public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOException { - // Fetch enough frames such that each tracker has one readable row. + // Fetch enough frames such that each tracker has one readable row (or is done). for (int i = 0; i < inputChannels.size(); i++) { final Tracker tracker = trackers.get(i); - if (tracker.isAtEndOfPushedData() && !pushNextFrame(i)) { + if (tracker.needsMoreDataForCurrentCursor() && !pushNextFrame(i)) { return nextAwait(); } } @@ -178,8 +180,8 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor startNewFrameIfNeeded(); while (!allTrackersAreAtEnd() - && !trackers.get(LEFT).needsMoreData() - && !trackers.get(RIGHT).needsMoreData()) { + && !trackers.get(LEFT).needsMoreDataForCurrentCursor() + && !trackers.get(RIGHT).needsMoreDataForCurrentCursor()) { // Algorithm can proceed: not all trackers are at the end of their streams, and no tracker needs more data to // read the current cursor or move it forward. if (nextIterationRunnable != null) { @@ -192,21 +194,12 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor // Two rows match if the keys compare equal _and_ neither key has a null component. (x JOIN y ON x.a = y.a does // not match rows where "x.a" is null.) - final boolean match = markCmp == 0 && trackers.get(LEFT).hasCompletelyNonNullMark(); + final boolean marksMatch = markCmp == 0 && trackers.get(LEFT).hasCompletelyNonNullMark(); - // If marked keys are equal on both sides ("match"), at least one side must have a complete set of rows - // for the marked key. - if (match && trackerWithCompleteSetForCurrentKey < 0) { - for (int i = 0; i < inputChannels.size(); i++) { - final Tracker tracker = trackers.get(i); - - // Fetch up to one frame from each tracker, to check if that tracker has a complete set. - // Can't fetch more than one frame, because channels are only guaranteed to have one frame per run. - if (tracker.hasCompleteSetForMark() || (pushNextFrame(i) && tracker.hasCompleteSetForMark())) { - trackerWithCompleteSetForCurrentKey = i; - break; - } - } + // If marked keys are equal on both sides ("marksMatch"), at least one side needs to have a complete set of rows + // for the marked key. Check if this is true, otherwise call nextAwait to read more data. + if (marksMatch && trackerWithCompleteSetForCurrentKey < 0) { + updateTrackerWithCompleteSetForCurrentKey(); if (trackerWithCompleteSetForCurrentKey < 0) { // Algorithm cannot proceed; fetch more frames on the next run. @@ -214,73 +207,13 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor } } - if (match || (markCmp <= 0 && joinType.isLefty()) || (markCmp >= 0 && joinType.isRighty())) { - // Emit row, if there's room in the current frameWriter. - joinColumnSelectorFactory.cmp = markCmp; - joinColumnSelectorFactory.match = match; - - if (!frameWriter.addSelection()) { - if (frameWriter.getNumRows() > 0) { - // Out of space in the current frame. Run again without moving cursors. - flushCurrentFrame(); - return ReturnOrAwait.runAgain(); - } else { - throw new FrameRowTooLargeException(frameWriterFactory.allocatorCapacity()); - } - } + // Emit row if there was a match. + if (!emitRowIfNeeded(markCmp, marksMatch)) { + return ReturnOrAwait.runAgain(); } // Advance one or both trackers. - if (match) { - // Matching keys. First advance the tracker with the complete set. - final Tracker tracker = trackers.get(trackerWithCompleteSetForCurrentKey); - final Tracker otherTracker = trackers.get(trackerWithCompleteSetForCurrentKey == LEFT ? RIGHT : LEFT); - - tracker.advance(); - if (!tracker.isCurrentSameKeyAsMark()) { - // Reached end of complete set. Advance the other tracker. - otherTracker.advance(); - - // On next iteration (when we're sure to have data) either rewind the complete-set tracker, or update marks - // of both, as appropriate. - onNextIteration(() -> { - if (otherTracker.isCurrentSameKeyAsMark()) { - otherTracker.markCurrent(); // Set mark to enable cleanup of old frames. - tracker.rewindToMark(); - } else { - // Reached end of the other side too. Advance marks on both trackers. - tracker.markCurrent(); - otherTracker.markCurrent(); - trackerWithCompleteSetForCurrentKey = -1; - } - }); - } - } else { - final int trackerToAdvance; - - if (markCmp < 0) { - trackerToAdvance = LEFT; - } else if (markCmp > 0) { - trackerToAdvance = RIGHT; - } else { - // Key is null on both sides. Note that there is a preference for running through the left side first - // on a FULL join. It doesn't really matter which side we run through first, but we do need to be consistent - // for the benefit of the logic in "shouldEmitColumnValue". - trackerToAdvance = joinType.isLefty() ? LEFT : RIGHT; - } - - final Tracker tracker = trackers.get(trackerToAdvance); - - tracker.advance(); - - // On next iteration (when we're sure to have data), update mark if the key changed. - onNextIteration(() -> { - if (!tracker.isCurrentSameKeyAsMark()) { - tracker.markCurrent(); - trackerWithCompleteSetForCurrentKey = -1; - } - }); - } + advanceTrackersAfterEmittingRow(markCmp, marksMatch); } if (allTrackersAreAtEnd()) { @@ -299,8 +232,152 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor } /** - * Returns a {@link ReturnOrAwait#awaitAll} for the channel numbers that need more data and have not yet hit their - * buffered-bytes limit, {@link Limits#MAX_BUFFERED_BYTES_FOR_SORT_MERGE_JOIN}. + * Set {@link #trackerWithCompleteSetForCurrentKey} to the lowest-numbered {@link Tracker} that has a complete + * set of rows available for its mark. + */ + private void updateTrackerWithCompleteSetForCurrentKey() + { + for (int i = 0; i < inputChannels.size(); i++) { + final Tracker tracker = trackers.get(i); + + // Fetch up to one frame from each tracker, to check if that tracker has a complete set. + // Can't fetch more than one frame, because channels are only guaranteed to have one frame per run. + if (tracker.hasCompleteSetForMark() || (pushNextFrame(i) && tracker.hasCompleteSetForMark())) { + trackerWithCompleteSetForCurrentKey = i; + return; + } + } + + trackerWithCompleteSetForCurrentKey = -1; + } + + /** + * Emits a joined row based on the current state of all trackers. + * + * @param markCmp result of {@link #compareMarks()} + * @param marksMatch whether the marks actually matched, taking nulls into account + * + * @return true if cursors should be advanced, false if we should run again without moving cursors + */ + private boolean emitRowIfNeeded(final int markCmp, final boolean marksMatch) throws IOException + { + if (marksMatch || (markCmp <= 0 && joinType.isLefty()) || (markCmp >= 0 && joinType.isRighty())) { + // Emit row, if there's room in the current frameWriter. + joinColumnSelectorFactory.cmp = markCmp; + joinColumnSelectorFactory.match = marksMatch; + + if (!frameWriter.addSelection()) { + if (frameWriter.getNumRows() > 0) { + // Out of space in the current frame. Run again without moving cursors. + flushCurrentFrame(); + return false; + } else { + throw new FrameRowTooLargeException(frameWriterFactory.allocatorCapacity()); + } + } + } + + return true; + } + + /** + * Advance one or both trackers after emitting a row. + * + * @param markCmp result of {@link #compareMarks()} + * @param marksMatch whether the marks actually matched, taking nulls into account + */ + private void advanceTrackersAfterEmittingRow(final int markCmp, final boolean marksMatch) + { + if (marksMatch) { + // Matching keys. First advance the tracker with the complete set. + final Tracker completeSetTracker = trackers.get(trackerWithCompleteSetForCurrentKey); + final Tracker otherTracker = trackers.get(trackerWithCompleteSetForCurrentKey == LEFT ? RIGHT : LEFT); + + completeSetTracker.advance(); + if (!completeSetTracker.isCurrentSameKeyAsMark()) { + // Reached end of complete set. Advance the other tracker. + otherTracker.advance(); + + // On next iteration (when we're sure to have data) either rewind the complete-set tracker, or update marks + // of both, as appropriate. + onNextIteration(() -> { + if (otherTracker.isCurrentSameKeyAsMark()) { + completeSetTracker.rewindToMark(); + } else { + // Reached end of the other side too. Advance marks on both trackers. + completeSetTracker.markCurrent(); + trackerWithCompleteSetForCurrentKey = -1; + } + + // Always update mark of the other tracker, to enable cleanup of old frames. It doesn't ever need to + // be rewound. + otherTracker.markCurrent(); + }); + } + } else { + // Keys don't match. Advance based on what kind of join this is. + final int trackerToAdvance; + final boolean skipMarkedKey; + + if (markCmp < 0) { + trackerToAdvance = LEFT; + } else if (markCmp > 0) { + trackerToAdvance = RIGHT; + } else { + // Key is null on both sides. Note that there is a preference for running through the left side first + // on a FULL join. It doesn't really matter which side we run through first, but we do need to be consistent + // for the benefit of the logic in "shouldEmitColumnValue". + trackerToAdvance = joinType.isLefty() ? LEFT : RIGHT; + } + + // Skip marked key entirely if we're on the "off" side of the join. (i.e., right side of a LEFT join.) + // Note that for FULL joins, entire keys are never skipped, because they are both lefty and righty. + if (trackerToAdvance == LEFT) { + skipMarkedKey = !joinType.isLefty(); + } else { + skipMarkedKey = !joinType.isRighty(); + } + + final Tracker tracker = trackers.get(trackerToAdvance); + + // Advance past marked key, or as far as we can. + boolean didKeyChange = false; + + do { + // Always advance a single row. If we're in "skipMarkedKey" mode, then we'll loop through later and + // potentially skip multiple rows with the same marked key. + tracker.advance(); + + if (tracker.isAtEndOfPushedData()) { + break; + } + + didKeyChange = !tracker.isCurrentSameKeyAsMark(); + + // Always update mark, even if key hasn't changed, to enable cleanup of old frames. + tracker.markCurrent(); + } while (skipMarkedKey && !didKeyChange); + + if (didKeyChange) { + trackerWithCompleteSetForCurrentKey = -1; + } else if (tracker.isAtEndOfPushedData()) { + // Not clear if we reached a new key or not. + // So, on next iteration (when we're sure to have data), check if we've moved on to a new key. + onNextIteration(() -> { + if (!tracker.isCurrentSameKeyAsMark()) { + trackerWithCompleteSetForCurrentKey = -1; + } + + // Always update mark, even if key hasn't changed, to enable cleanup of old frames. + tracker.markCurrent(); + }); + } + } + } + + /** + * Returns a {@link ReturnOrAwait#awaitAll} for channels where {@link Tracker#needsMoreDataForCurrentCursor()} + * and {@link Tracker#canBufferMoreFrames()}. * * If all channels have hit their limit, throws {@link MSQException} with {@link TooManyRowsWithSameKeyFault}. */ @@ -309,10 +386,11 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor final IntSet awaitSet = new IntOpenHashSet(); int trackerAtLimit = -1; + // Add all trackers that "needsMoreData" to awaitSet. for (int i = 0; i < inputChannels.size(); i++) { final Tracker tracker = trackers.get(i); - if (tracker.needsMoreData()) { - if (tracker.totalBytesBuffered() < Limits.MAX_BUFFERED_BYTES_FOR_SORT_MERGE_JOIN) { + if (tracker.needsMoreDataForCurrentCursor()) { + if (tracker.canBufferMoreFrames()) { awaitSet.add(i); } else if (trackerAtLimit < 0) { trackerAtLimit = i; @@ -320,19 +398,31 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor } } - if (awaitSet.isEmpty() && trackerAtLimit > 0) { + if (awaitSet.isEmpty()) { + // No tracker reported that it "needsMoreData" to read the current cursor. However, we may still need to read + // more data to have a complete set for the current mark. + for (int i = 0; i < inputChannels.size(); i++) { + final Tracker tracker = trackers.get(i); + if (!tracker.hasCompleteSetForMark()) { + if (tracker.canBufferMoreFrames()) { + awaitSet.add(i); + } else if (trackerAtLimit < 0) { + trackerAtLimit = i; + } + } + } + } + + if (awaitSet.isEmpty() && trackerAtLimit >= 0) { // All trackers that need more data are at their max buffered bytes limit. Generate a nice exception. final Tracker tracker = trackers.get(trackerAtLimit); - if (tracker.totalBytesBuffered() > Limits.MAX_BUFFERED_BYTES_FOR_SORT_MERGE_JOIN) { - // Generate a nice exception. - throw new MSQException( - new TooManyRowsWithSameKeyFault( - tracker.readMarkKey(), - tracker.totalBytesBuffered(), - Limits.MAX_BUFFERED_BYTES_FOR_SORT_MERGE_JOIN - ) - ); - } + throw new MSQException( + new TooManyRowsWithSameKeyFault( + tracker.readMarkKey(), + tracker.totalBytesBuffered(), + maxBufferedBytes + ) + ); } return ReturnOrAwait.awaitAll(awaitSet); @@ -353,7 +443,13 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor } /** - * Compares the marked rows of the two {@link #trackers}. + * Compares the marked rows of the two {@link #trackers}. This method returns 0 if both sides are null, even + * though this is not considered a match by join semantics. Therefore, it is important to also check + * {@link Tracker#hasCompletelyNonNullMark()}. + * + * @return negative if {@link #LEFT} key is earlier, positive if {@link #RIGHT} key is earlier, zero if the keys + * are the same. Returns zero even if a key component is null, even though this is not considered a match by + * join semantics. * * @throws IllegalStateException if either tracker does not have a marked row and is not completely done */ @@ -394,6 +490,8 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor } else if (channel.isFinished()) { tracker.push(null); return true; + } else if (!tracker.canBufferMoreFrames()) { + return false; } else { final Frame frame = channel.read(); @@ -450,6 +548,7 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor private final List holders = new ArrayList<>(); private final ReadableInput input; private final List keyColumns; + private final long maxBytesBuffered; // markFrame and markRow are the first frame and row with the current key. private int markFrame = -1; @@ -461,10 +560,11 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor // done indicates that no more data is available in the channel. private boolean done; - public Tracker(ReadableInput input, List keyColumns) + public Tracker(ReadableInput input, List keyColumns, long maxBytesBuffered) { this.input = input; this.keyColumns = keyColumns; + this.maxBytesBuffered = maxBytesBuffered; } /** @@ -533,6 +633,16 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor return bytes; } + /** + * Whether this tracker can accept more frames without exceeding {@link #maxBufferedBytes}. Always returns true + * if the number of buffered frames is zero or one, because the join algorithm may require two frames being + * buffered. (For example, if we need to verify that the last row in a frame contains a complete set of a key.) + */ + public boolean canBufferMoreFrames() + { + return holders.size() <= 1 || totalBytesBuffered() < maxBytesBuffered; + } + /** * Cursor containing the current row. */ @@ -655,7 +765,7 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor /** * Whether this tracker needs more data in order to read the current cursor location or move it forward. */ - public boolean needsMoreData() + public boolean needsMoreDataForCurrentCursor() { return !done && isAtEndOfPushedData(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorFactory.java index 9aa50630929..76e05d3ce0c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorFactory.java @@ -180,7 +180,8 @@ public class SortMergeJoinFrameProcessorFactory extends BaseFrameProcessorFactor stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator()), rightPrefix, keyColumns, - joinType + joinType, + frameContext.memoryParameters().getSortMergeJoinMemory() ); } ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java index 78ecacbef26..29614fc0734 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java @@ -32,11 +32,11 @@ public class WorkerMemoryParametersTest @Test public void test_oneWorkerInJvm_alone() { - Assert.assertEquals(params(1, 41, 224_785_000, 100_650_000, 75_000_000), create(1_000_000_000, 1, 1, 1, 0, 0)); - Assert.assertEquals(params(2, 13, 149_410_000, 66_900_000, 75_000_000), create(1_000_000_000, 1, 2, 1, 0, 0)); - Assert.assertEquals(params(4, 3, 89_110_000, 39_900_000, 75_000_000), create(1_000_000_000, 1, 4, 1, 0, 0)); - Assert.assertEquals(params(3, 2, 48_910_000, 21_900_000, 75_000_000), create(1_000_000_000, 1, 8, 1, 0, 0)); - Assert.assertEquals(params(2, 2, 33_448_460, 14_976_922, 75_000_000), create(1_000_000_000, 1, 12, 1, 0, 0)); + Assert.assertEquals(params(335_500_000, 1, 41, 75_000_000), create(1_000_000_000, 1, 1, 1, 0, 0)); + Assert.assertEquals(params(223_000_000, 2, 13, 75_000_000), create(1_000_000_000, 1, 2, 1, 0, 0)); + Assert.assertEquals(params(133_000_000, 4, 3, 75_000_000), create(1_000_000_000, 1, 4, 1, 0, 0)); + Assert.assertEquals(params(73_000_000, 3, 2, 75_000_000), create(1_000_000_000, 1, 8, 1, 0, 0)); + Assert.assertEquals(params(49_923_076, 2, 2, 75_000_000), create(1_000_000_000, 1, 12, 1, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, @@ -54,8 +54,8 @@ public class WorkerMemoryParametersTest @Test public void test_oneWorkerInJvm_twoHundredWorkersInCluster() { - Assert.assertEquals(params(1, 83, 317_580_000, 142_200_000, 150_000_000), create(2_000_000_000, 1, 1, 200, 0, 0)); - Assert.assertEquals(params(2, 27, 166_830_000, 74_700_000, 150_000_000), create(2_000_000_000, 1, 2, 200, 0, 0)); + Assert.assertEquals(params(474_000_000, 1, 83, 150_000_000), create(2_000_000_000, 1, 1, 200, 0, 0)); + Assert.assertEquals(params(249_000_000, 2, 27, 150_000_000), create(2_000_000_000, 1, 2, 200, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, @@ -68,11 +68,11 @@ public class WorkerMemoryParametersTest @Test public void test_fourWorkersInJvm_twoHundredWorkersInCluster() { - Assert.assertEquals(params(1, 150, 679_380_000, 304_200_000, 168_750_000), create(9_000_000_000L, 4, 1, 200, 0, 0)); - Assert.assertEquals(params(2, 62, 543_705_000, 243_450_000, 168_750_000), create(9_000_000_000L, 4, 2, 200, 0, 0)); - Assert.assertEquals(params(4, 22, 374_111_250, 167_512_500, 168_750_000), create(9_000_000_000L, 4, 4, 200, 0, 0)); - Assert.assertEquals(params(4, 14, 204_517_500, 91_575_000, 168_750_000), create(9_000_000_000L, 4, 8, 200, 0, 0)); - Assert.assertEquals(params(4, 8, 68_842_500, 30_825_000, 168_750_000), create(9_000_000_000L, 4, 16, 200, 0, 0)); + Assert.assertEquals(params(1_014_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 200, 0, 0)); + Assert.assertEquals(params(811_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 200, 0, 0)); + Assert.assertEquals(params(558_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 200, 0, 0)); + Assert.assertEquals(params(305_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 200, 0, 0)); + Assert.assertEquals(params(102_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 200, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, @@ -82,7 +82,7 @@ public class WorkerMemoryParametersTest Assert.assertEquals(new TooManyWorkersFault(200, 124), e.getFault()); // Make sure 124 actually works, and 125 doesn't. (Verify the error message above.) - Assert.assertEquals(params(4, 3, 16_750_000, 7_500_000, 150_000_000), create(8_000_000_000L, 4, 32, 124, 0, 0)); + Assert.assertEquals(params(25_000_000, 4, 3, 150_000_000), create(8_000_000_000L, 4, 32, 124, 0, 0)); final MSQException e2 = Assert.assertThrows( MSQException.class, @@ -96,8 +96,8 @@ public class WorkerMemoryParametersTest public void test_oneWorkerInJvm_smallWorkerCapacity() { // Supersorter max channels per processer are one less than they are usually to account for extra frames that are required while creating composing output channels - Assert.assertEquals(params(1, 3, 27_604_000, 12_360_000, 9_600_000), create(128_000_000, 1, 1, 1, 0, 0)); - Assert.assertEquals(params(1, 1, 17_956_000, 8_040_000, 9_600_000), create(128_000_000, 1, 2, 1, 0, 0)); + Assert.assertEquals(params(41_200_000, 1, 3, 9_600_000), create(128_000_000, 1, 1, 1, 0, 0)); + Assert.assertEquals(params(26_800_000, 1, 1, 9_600_000), create(128_000_000, 1, 2, 1, 0, 0)); final MSQException e = Assert.assertThrows( MSQException.class, @@ -120,14 +120,10 @@ public class WorkerMemoryParametersTest @Test public void test_fourWorkersInJvm_twoHundredWorkersInCluster_hashPartitions() { - Assert.assertEquals( - params(1, 150, 545_380_000, 244_200_000, 168_750_000), create(9_000_000_000L, 4, 1, 200, 200, 0)); - Assert.assertEquals( - params(2, 62, 409_705_000, 183_450_000, 168_750_000), create(9_000_000_000L, 4, 2, 200, 200, 0)); - Assert.assertEquals( - params(4, 22, 240_111_250, 107_512_500, 168_750_000), create(9_000_000_000L, 4, 4, 200, 200, 0)); - Assert.assertEquals( - params(4, 14, 70_517_500, 31_575_000, 168_750_000), create(9_000_000_000L, 4, 8, 200, 200, 0)); + Assert.assertEquals(params(814_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 200, 200, 0)); + Assert.assertEquals(params(611_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 200, 200, 0)); + Assert.assertEquals(params(358_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 200, 200, 0)); + Assert.assertEquals(params(105_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 200, 200, 0)); final MSQException e = Assert.assertThrows( MSQException.class, @@ -137,7 +133,7 @@ public class WorkerMemoryParametersTest Assert.assertEquals(new TooManyWorkersFault(200, 138), e.getFault()); // Make sure 138 actually works, and 139 doesn't. (Verify the error message above.) - Assert.assertEquals(params(4, 8, 17_922_500, 8_025_000, 168_750_000), create(9_000_000_000L, 4, 16, 138, 138, 0)); + Assert.assertEquals(params(26_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 138, 138, 0)); final MSQException e2 = Assert.assertThrows( MSQException.class, @@ -165,18 +161,16 @@ public class WorkerMemoryParametersTest } private static WorkerMemoryParameters params( + final long processorBundleMemory, final int superSorterMaxActiveProcessors, final int superSorterMaxChannelsPerProcessor, - final long appenderatorMemory, - final long broadcastJoinMemory, final int partitionStatisticsMaxRetainedBytes ) { return new WorkerMemoryParameters( + processorBundleMemory, superSorterMaxActiveProcessors, superSorterMaxChannelsPerProcessor, - appenderatorMemory, - broadcastJoinMemory, partitionStatisticsMaxRetainedBytes ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java index cfc74d792f8..060b14cec12 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java @@ -20,8 +20,10 @@ package org.apache.druid.msq.querykit.common; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.primitives.Ints; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.common.config.NullHandling; @@ -46,6 +48,8 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.TooManyRowsWithSameKeyFault; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.StagePartition; @@ -58,8 +62,12 @@ import org.apache.druid.segment.join.JoinTestHelper; import org.apache.druid.segment.join.JoinType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.hamcrest.Matchers; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -79,6 +87,7 @@ import java.util.concurrent.TimeUnit; public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest { private static final StagePartition STAGE_PARTITION = new StagePartition(new StageId("q", 0), 0); + private static final long MAX_BUFFERED_BYTES = 10_000_000; private final int rowsPerInputFrame; private final int rowsPerOutputFrame; @@ -154,7 +163,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)), ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)) ), - JoinType.LEFT + JoinType.LEFT, + MAX_BUFFERED_BYTES ); assertResult(processor, outputChannel.readable(), joinSignature, Collections.emptyList()); @@ -198,7 +208,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)), ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)) ), - JoinType.LEFT + JoinType.LEFT, + MAX_BUFFERED_BYTES ); final List> expectedRows = Arrays.asList( @@ -273,7 +284,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)), ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)) ), - JoinType.INNER + JoinType.INNER, + MAX_BUFFERED_BYTES ); assertResult(processor, outputChannel.readable(), joinSignature, Collections.emptyList()); @@ -313,7 +325,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)), ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)) ), - JoinType.LEFT + JoinType.LEFT, + MAX_BUFFERED_BYTES ); final List> expectedRows = Arrays.asList( @@ -383,7 +396,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest makeFrameWriterFactory(joinSignature), "j0.", ImmutableList.of(Collections.emptyList(), Collections.emptyList()), - JoinType.INNER + JoinType.INNER, + MAX_BUFFERED_BYTES ); final List> expectedRows = Arrays.asList( @@ -495,7 +509,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest new KeyColumn("regionIsoCode", KeyOrder.ASCENDING) ) ), - JoinType.LEFT + JoinType.LEFT, + MAX_BUFFERED_BYTES ); final List> expectedRows = Arrays.asList( @@ -573,7 +588,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest ImmutableList.of(new KeyColumn("regionIsoCode", KeyOrder.ASCENDING)), ImmutableList.of(new KeyColumn("regionIsoCode", KeyOrder.ASCENDING)) ), - JoinType.RIGHT + JoinType.RIGHT, + MAX_BUFFERED_BYTES ); final List> expectedRows = Arrays.asList( @@ -654,7 +670,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest ImmutableList.of(new KeyColumn("regionIsoCode", KeyOrder.ASCENDING)), ImmutableList.of(new KeyColumn("regionIsoCode", KeyOrder.ASCENDING)) ), - JoinType.FULL + JoinType.FULL, + MAX_BUFFERED_BYTES ); final List> expectedRows = Arrays.asList( @@ -732,7 +749,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest ImmutableList.of(new KeyColumn("countryNumber", KeyOrder.ASCENDING)), ImmutableList.of(new KeyColumn("countryNumber", KeyOrder.ASCENDING)) ), - JoinType.LEFT + JoinType.LEFT, + MAX_BUFFERED_BYTES ); final String countryCodeForNull; @@ -825,7 +843,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest ImmutableList.of(new KeyColumn("countryNumber", KeyOrder.ASCENDING)), ImmutableList.of(new KeyColumn("countryNumber", KeyOrder.ASCENDING)) ), - JoinType.RIGHT + JoinType.RIGHT, + MAX_BUFFERED_BYTES ); final String countryCodeForNull; @@ -918,7 +937,8 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)), ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)) ), - JoinType.INNER + JoinType.INNER, + MAX_BUFFERED_BYTES ); final List> expectedRows = Arrays.asList( @@ -950,6 +970,234 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest assertResult(processor, outputChannel.readable(), joinSignature, expectedRows); } + @Test + public void testInnerJoinCountryIsoCode_withMaxBufferedBytesLimit_succeeds() throws Exception + { + final ReadableInput factChannel = buildFactInput( + ImmutableList.of( + new KeyColumn("countryIsoCode", KeyOrder.ASCENDING), + new KeyColumn("page", KeyOrder.ASCENDING) + ) + ); + + final ReadableInput countriesChannel = + buildCountriesInput(ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING))); + + final BlockingQueueFrameChannel outputChannel = BlockingQueueFrameChannel.minimal(); + + final RowSignature joinSignature = + RowSignature.builder() + .add("page", ColumnType.STRING) + .add("countryIsoCode", ColumnType.STRING) + .add("j0.countryIsoCode", ColumnType.STRING) + .add("j0.countryName", ColumnType.STRING) + .add("j0.countryNumber", ColumnType.LONG) + .build(); + + final SortMergeJoinFrameProcessor processor = new SortMergeJoinFrameProcessor( + factChannel, + countriesChannel, + outputChannel.writable(), + makeFrameWriterFactory(joinSignature), + "j0.", + ImmutableList.of( + ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)), + ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)) + ), + JoinType.INNER, + 1 + ); + + final List> expectedRows = Arrays.asList( + Arrays.asList("Peremptory norm", "AU", "AU", "Australia", 0L), + Arrays.asList("Didier Leclair", "CA", "CA", "Canada", 1L), + Arrays.asList("Les Argonautes", "CA", "CA", "Canada", 1L), + Arrays.asList("Sarah Michelle Gellar", "CA", "CA", "Canada", 1L), + Arrays.asList("Golpe de Estado en Chile de 1973", "CL", "CL", "Chile", 2L), + Arrays.asList("Diskussion:Sebastian Schulz", "DE", "DE", "Germany", 3L), + Arrays.asList("Gabinete Ministerial de Rafael Correa", "EC", "EC", "Ecuador", 4L), + Arrays.asList("Saison 9 de Secret Story", "FR", "FR", "France", 5L), + Arrays.asList("Glasgow", "GB", "GB", "United Kingdom", 6L), + Arrays.asList("Giusy Ferreri discography", "IT", "IT", "Italy", 7L), + Arrays.asList("Roma-Bangkok", "IT", "IT", "Italy", 7L), + Arrays.asList("青野武", "JP", "JP", "Japan", 8L), + Arrays.asList("유희왕 GX", "KR", "KR", "Republic of Korea", 9L), + Arrays.asList("History of Fourems", "MMMM", "MMMM", "Fourems", 205L), + Arrays.asList("Mathis Bolly", "MX", "MX", "Mexico", 10L), + Arrays.asList("Алиса в Зазеркалье", "NO", "NO", "Norway", 11L), + Arrays.asList("Cream Soda", "SU", "SU", "States United", 15L), + Arrays.asList("Wendigo", "SV", "SV", "El Salvador", 12L), + Arrays.asList("Carlo Curti", "US", "US", "United States", 13L), + Arrays.asList("DirecTV", "US", "US", "United States", 13L), + Arrays.asList("Old Anatolian Turkish", "US", "US", "United States", 13L), + Arrays.asList("Otjiwarongo Airport", "US", "US", "United States", 13L), + Arrays.asList("President of India", "US", "US", "United States", 13L) + ); + + assertResult(processor, outputChannel.readable(), joinSignature, expectedRows); + } + + @Test + public void testInnerJoinCountryIsoCode_backwards_withMaxBufferedBytesLimit_succeeds() throws Exception + { + final ReadableInput factChannel = buildFactInput( + ImmutableList.of( + new KeyColumn("countryIsoCode", KeyOrder.ASCENDING), + new KeyColumn("page", KeyOrder.ASCENDING) + ) + ); + + final ReadableInput countriesChannel = + buildCountriesInput(ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING))); + + final BlockingQueueFrameChannel outputChannel = BlockingQueueFrameChannel.minimal(); + + final RowSignature joinSignature = + RowSignature.builder() + .add("j0.page", ColumnType.STRING) + .add("j0.countryIsoCode", ColumnType.STRING) + .add("countryIsoCode", ColumnType.STRING) + .add("countryName", ColumnType.STRING) + .add("countryNumber", ColumnType.LONG) + .build(); + + final SortMergeJoinFrameProcessor processor = new SortMergeJoinFrameProcessor( + countriesChannel, + factChannel, + outputChannel.writable(), + makeFrameWriterFactory(joinSignature), + "j0.", + ImmutableList.of( + ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)), + ImmutableList.of(new KeyColumn("countryIsoCode", KeyOrder.ASCENDING)) + ), + JoinType.INNER, + 1 + ); + + final List> expectedRows = Arrays.asList( + Arrays.asList("Peremptory norm", "AU", "AU", "Australia", 0L), + Arrays.asList("Didier Leclair", "CA", "CA", "Canada", 1L), + Arrays.asList("Les Argonautes", "CA", "CA", "Canada", 1L), + Arrays.asList("Sarah Michelle Gellar", "CA", "CA", "Canada", 1L), + Arrays.asList("Golpe de Estado en Chile de 1973", "CL", "CL", "Chile", 2L), + Arrays.asList("Diskussion:Sebastian Schulz", "DE", "DE", "Germany", 3L), + Arrays.asList("Gabinete Ministerial de Rafael Correa", "EC", "EC", "Ecuador", 4L), + Arrays.asList("Saison 9 de Secret Story", "FR", "FR", "France", 5L), + Arrays.asList("Glasgow", "GB", "GB", "United Kingdom", 6L), + Arrays.asList("Giusy Ferreri discography", "IT", "IT", "Italy", 7L), + Arrays.asList("Roma-Bangkok", "IT", "IT", "Italy", 7L), + Arrays.asList("青野武", "JP", "JP", "Japan", 8L), + Arrays.asList("유희왕 GX", "KR", "KR", "Republic of Korea", 9L), + Arrays.asList("History of Fourems", "MMMM", "MMMM", "Fourems", 205L), + Arrays.asList("Mathis Bolly", "MX", "MX", "Mexico", 10L), + Arrays.asList("Алиса в Зазеркалье", "NO", "NO", "Norway", 11L), + Arrays.asList("Cream Soda", "SU", "SU", "States United", 15L), + Arrays.asList("Wendigo", "SV", "SV", "El Salvador", 12L), + Arrays.asList("Carlo Curti", "US", "US", "United States", 13L), + Arrays.asList("DirecTV", "US", "US", "United States", 13L), + Arrays.asList("Old Anatolian Turkish", "US", "US", "United States", 13L), + Arrays.asList("Otjiwarongo Airport", "US", "US", "United States", 13L), + Arrays.asList("President of India", "US", "US", "United States", 13L) + ); + + assertResult(processor, outputChannel.readable(), joinSignature, expectedRows); + } + + @Test + public void testCountrySelfJoin() throws Exception + { + final ReadableInput factChannel1 = buildFactInput(ImmutableList.of(new KeyColumn("channel", KeyOrder.ASCENDING))); + final ReadableInput factChannel2 = buildFactInput(ImmutableList.of(new KeyColumn("channel", KeyOrder.ASCENDING))); + + final BlockingQueueFrameChannel outputChannel = BlockingQueueFrameChannel.minimal(); + + final RowSignature joinSignature = + RowSignature.builder() + .add("channel", ColumnType.STRING) + .build(); + + final SortMergeJoinFrameProcessor processor = new SortMergeJoinFrameProcessor( + factChannel1, + factChannel2, + outputChannel.writable(), + makeFrameWriterFactory(joinSignature), + "j0.", + ImmutableList.of( + ImmutableList.of(new KeyColumn("channel", KeyOrder.ASCENDING)), + ImmutableList.of(new KeyColumn("channel", KeyOrder.ASCENDING)) + ), + JoinType.INNER, + MAX_BUFFERED_BYTES + ); + + final List> expectedRows = new ArrayList<>(); + + final ImmutableMap expectedCounts = + ImmutableMap.builder() + .put("#ca.wikipedia", 1L) + .put("#de.wikipedia", 1L) + .put("#en.wikipedia", 196L) + .put("#es.wikipedia", 16L) + .put("#fr.wikipedia", 9L) + .put("#ja.wikipedia", 1L) + .put("#ko.wikipedia", 1L) + .put("#ru.wikipedia", 1L) + .put("#vi.wikipedia", 9L) + .build(); + + for (final Map.Entry entry : expectedCounts.entrySet()) { + for (int i = 0; i < Ints.checkedCast(entry.getValue()); i++) { + expectedRows.add(Collections.singletonList(entry.getKey())); + } + } + + assertResult(processor, outputChannel.readable(), joinSignature, expectedRows); + } + + @Test + public void testCountrySelfJoin_withMaxBufferedBytesLimit_fails() throws Exception + { + // Test is only valid when rowsPerInputFrame is low enough that we get multiple frames. + Assume.assumeThat(rowsPerInputFrame, Matchers.lessThanOrEqualTo(7)); + + final ReadableInput factChannel1 = buildFactInput(ImmutableList.of(new KeyColumn("channel", KeyOrder.ASCENDING))); + final ReadableInput factChannel2 = buildFactInput(ImmutableList.of(new KeyColumn("channel", KeyOrder.ASCENDING))); + + final BlockingQueueFrameChannel outputChannel = BlockingQueueFrameChannel.minimal(); + + final RowSignature joinSignature = + RowSignature.builder() + .add("channel", ColumnType.STRING) + .build(); + + final SortMergeJoinFrameProcessor processor = new SortMergeJoinFrameProcessor( + factChannel1, + factChannel2, + outputChannel.writable(), + makeFrameWriterFactory(joinSignature), + "j0.", + ImmutableList.of( + ImmutableList.of(new KeyColumn("channel", KeyOrder.ASCENDING)), + ImmutableList.of(new KeyColumn("channel", KeyOrder.ASCENDING)) + ), + JoinType.INNER, + 1 + ); + + final RuntimeException e = Assert.assertThrows( + RuntimeException.class, + () -> run(processor, outputChannel.readable(), joinSignature) + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RuntimeException.class)); + MatcherAssert.assertThat(e.getCause().getCause(), CoreMatchers.instanceOf(MSQException.class)); + MatcherAssert.assertThat( + ((MSQException) e.getCause().getCause()).getFault(), + CoreMatchers.instanceOf(TooManyRowsWithSameKeyFault.class) + ); + } + private void assertResult( final SortMergeJoinFrameProcessor processor, final ReadableFrameChannel readableOutputChannel, @@ -957,14 +1205,25 @@ public class SortMergeJoinFrameProcessorTest extends InitializedNullHandlingTest final List> expectedRows ) { - final ListenableFuture retVal = exec.runFully(processor, null); + final List> rowsFromProcessor = run(processor, readableOutputChannel, joinSignature); + FrameTestUtil.assertRowsEqual(Sequences.simple(expectedRows), Sequences.simple(rowsFromProcessor)); + } + + private List> run( + final SortMergeJoinFrameProcessor processor, + final ReadableFrameChannel readableOutputChannel, + final RowSignature joinSignature + ) + { + final ListenableFuture retValFromProcessor = exec.runFully(processor, null); final Sequence> rowsFromProcessor = FrameTestUtil.readRowsFromFrameChannel( readableOutputChannel, FrameReader.create(joinSignature) ); - FrameTestUtil.assertRowsEqual(Sequences.simple(expectedRows), rowsFromProcessor); - Assert.assertEquals(0L, (long) FutureUtils.getUnchecked(retVal, true)); + final List> rows = rowsFromProcessor.toList(); + Assert.assertEquals(0L, (long) FutureUtils.getUnchecked(retValFromProcessor, true)); + return rows; } private ReadableInput buildFactInput(final List keyColumns) throws IOException From 277aaa5c572725b7cacb4e7be36bad164a50f889 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 2 Jul 2023 19:37:15 -0700 Subject: [PATCH 57/74] remove druid.processing.columnCache.sizeBytes and CachingIndexed, combine string column implementations (#14500) * combine string column implementations changes: * generic indexed, front-coded, and auto string columns now all share the same column and index supplier implementations * remove CachingIndexed implementation, which I think is largely no longer needed by the switch of many things to directly using ByteBuffer, avoiding the cost of creating Strings * remove ColumnConfig.columnCacheSizeBytes since CachingIndexed was the only user --- .../druid/benchmark/BoundFilterBenchmark.java | 8 +- ...ryEncodedStringIndexSupplierBenchmark.java | 11 +- .../DimensionPredicateFilterBenchmark.java | 9 +- .../benchmark/FilterPartitionBenchmark.java | 5 - .../FilteredAggregatorBenchmark.java | 5 - .../GroupByTypeInterfaceBenchmark.java | 5 - .../druid/benchmark/InFilterBenchmark.java | 9 +- .../benchmark/JoinAndLookupBenchmark.java | 6 +- .../druid/benchmark/LikeFilterBenchmark.java | 9 +- .../benchmark/TopNTypeInterfaceBenchmark.java | 5 - ...arLongsEncodeDataFromSegmentBenchmark.java | 6 +- .../indexing/IndexMergeBenchmark.java | 6 +- .../indexing/IndexPersistBenchmark.java | 6 +- .../benchmark/query/GroupByBenchmark.java | 5 - .../druid/benchmark/query/ScanBenchmark.java | 6 +- .../benchmark/query/SearchBenchmark.java | 5 - .../benchmark/query/TimeseriesBenchmark.java | 5 - .../druid/benchmark/query/TopNBenchmark.java | 5 - .../timecompare/TimeCompareBenchmark.java | 5 - docs/configuration/index.md | 4 - .../msq/indexing/WorkerChatHandlerTest.java | 3 +- .../druid/msq/test/CalciteMSQTestsHelper.java | 3 +- .../apache/druid/msq/test/MSQTestBase.java | 3 +- .../druid/msq/test/MSQTestWorkerContext.java | 6 +- .../druid/indexing/common/TestUtils.java | 6 +- .../common/task/AppenderatorsTest.java | 6 +- .../common/task/BatchAppenderatorsTest.java | 8 +- .../common/task/CompactionTaskTest.java | 3 +- pom.xml | 4 +- .../druid/query/DruidProcessingConfig.java | 7 - .../org/apache/druid/segment/IndexIO.java | 27 +- .../apache/druid/segment/MMappedIndex.java | 8 - .../druid/segment/column/ColumnConfig.java | 18 +- .../IndexedStringDruidPredicateIndex.java | 29 - .../IndexedUtf8LexicographicalRangeIndex.java | 17 - .../column/StringDictionaryEncodedColumn.java | 807 ------------------ .../StringUtf8DictionaryEncodedColumn.java | 323 ++++++- .../druid/segment/data/CachingIndexed.java | 158 ---- .../NestedFieldDictionaryEncodedColumn.java | 6 +- .../ScalarStringColumnAndIndexSupplier.java | 115 +-- .../druid/segment/nested/VariantColumn.java | 4 +- .../DictionaryEncodedColumnPartSerde.java | 123 +-- .../DictionaryEncodedColumnSupplier.java | 100 --- .../DictionaryEncodedStringIndexSupplier.java | 127 --- .../NestedCommonFormatColumnPartSerde.java | 223 +++-- ...ava => StringUtf8ColumnIndexSupplier.java} | 38 +- ...gUtf8DictionaryEncodedColumnSupplier.java} | 17 +- .../query/DruidProcessingConfigTest.java | 4 - .../aggregation/AggregationTestHelper.java | 20 - ...ByLimitPushDownInsufficientBufferTest.java | 5 - ...roupByLimitPushDownMultiNodeMergeTest.java | 5 - .../groupby/GroupByMultiSegmentTest.java | 5 - .../groupby/NestedQueryPushDownTest.java | 6 +- .../query/metadata/SegmentAnalyzerTest.java | 4 +- ...ColumnSelectorColumnIndexSelectorTest.java | 4 +- .../segment/CustomSegmentizerFactoryTest.java | 3 +- .../apache/druid/segment/IndexBuilder.java | 4 +- .../IndexIONullColumnsCompatibilityTest.java | 2 +- .../druid/segment/IndexMergerTestBase.java | 6 +- .../org/apache/druid/segment/TestHelper.java | 22 +- .../druid/segment/V9IndexLoaderTest.java | 3 +- .../filter/ExtractionDimFilterTest.java | 7 +- .../PredicateValueMatcherFactoryTest.java | 18 +- .../segment/filter/ValueMatchersTest.java | 32 +- .../druid/segment/join/JoinTestHelper.java | 2 +- .../BroadcastSegmentIndexedTableTest.java | 3 +- ...MappedQueryableSegmentizerFactoryTest.java | 3 +- .../loading/SegmentizerFactoryTest.java | 5 - .../nested/NestedDataColumnSupplierTest.java | 5 - .../NestedFieldColumnIndexSupplierTest.java | 12 - ...tionaryEncodedStringIndexSupplierTest.java | 9 +- .../serde/NullColumnPartSerdeTest.java | 16 +- .../virtual/DummyStringVirtualColumn.java | 12 +- .../LocalDataStorageDruidModuleTest.java | 2 +- ...edSegmensSinksBatchAppenderatorTester.java | 6 +- ...DefaultOfflineAppenderatorFactoryTest.java | 5 - ...enAndClosedSegmentsAppenderatorTester.java | 5 - .../StreamAppenderatorTester.java | 5 - ...tManagerBroadcastJoinIndexedTableTest.java | 3 +- .../SegmentManagerThreadSafetyTest.java | 3 +- .../org/apache/druid/cli/DumpSegment.java | 5 - .../apache/druid/cli/ValidateSegments.java | 5 - 82 files changed, 669 insertions(+), 1901 deletions(-) delete mode 100644 processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java delete mode 100644 processing/src/main/java/org/apache/druid/segment/data/CachingIndexed.java delete mode 100644 processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java delete mode 100644 processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplier.java rename processing/src/main/java/org/apache/druid/segment/serde/{StringFrontCodedColumnIndexSupplier.java => StringUtf8ColumnIndexSupplier.java} (80%) rename processing/src/main/java/org/apache/druid/segment/serde/{StringFrontCodedDictionaryEncodedColumnSupplier.java => StringUtf8DictionaryEncodedColumnSupplier.java} (80%) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/BoundFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/BoundFilterBenchmark.java index 8f242c40f2f..e1dd7fe4f2f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/BoundFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/BoundFilterBenchmark.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.filter.BoundFilter; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier; +import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -161,10 +161,6 @@ public class BoundFilterBenchmark final BitmapFactory bitmapFactory = new RoaringBitmapFactory(); final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance(); final List ints = generateInts(); - final GenericIndexed dictionary = GenericIndexed.fromIterable( - FluentIterable.from(ints).transform(Object::toString), - GenericIndexed.STRING_STRATEGY - ); final GenericIndexed bitmaps = GenericIndexed.fromIterable( FluentIterable.from(ints) .transform( @@ -183,7 +179,7 @@ public class BoundFilterBenchmark ); selector = new MockColumnIndexSelector( bitmapFactory, - new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null) + new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null) ); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java index cc3f68aba11..73cba8c5d97 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java @@ -32,7 +32,7 @@ import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; -import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier; +import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -93,11 +93,6 @@ public class DictionaryEncodedStringIndexSupplierBenchmark final BitmapFactory bitmapFactory = new RoaringBitmapFactory(); final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance(); final Iterable ints = intGenerator(); - final GenericIndexed dictionary = GenericIndexed.fromIterable( - FluentIterable.from(ints) - .transform(Object::toString), - GenericIndexed.STRING_STRATEGY - ); final GenericIndexed dictionaryUtf8 = GenericIndexed.fromIterable( FluentIterable.from(ints) .transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))), @@ -115,8 +110,8 @@ public class DictionaryEncodedStringIndexSupplierBenchmark .iterator(), serdeFactory.getObjectStrategy() ); - DictionaryEncodedStringIndexSupplier indexSupplier = - new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null); + StringUtf8ColumnIndexSupplier indexSupplier = + new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null); stringValueSetIndex = (IndexedUtf8ValueSetIndex) indexSupplier.as(StringValueSetIndex.class); List filterValues = new ArrayList<>(); List nonFilterValues = new ArrayList<>(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/DimensionPredicateFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/DimensionPredicateFilterBenchmark.java index 810ad4219c2..602f838680b 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/DimensionPredicateFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DimensionPredicateFilterBenchmark.java @@ -38,7 +38,7 @@ import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.filter.DimensionPredicateFilter; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier; +import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -122,11 +122,6 @@ public class DimensionPredicateFilterBenchmark final BitmapFactory bitmapFactory = new RoaringBitmapFactory(); final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance(); final List ints = generateInts(); - final GenericIndexed dictionary = GenericIndexed.fromIterable( - FluentIterable.from(ints) - .transform(Object::toString), - GenericIndexed.STRING_STRATEGY - ); final GenericIndexed dictionaryUtf8 = GenericIndexed.fromIterable( FluentIterable.from(ints) .transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))), @@ -145,7 +140,7 @@ public class DimensionPredicateFilterBenchmark ); selector = new MockColumnIndexSelector( bitmapFactory, - new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null) + new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null) ); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java index 775aa0bb1ed..bc5c79c0af3 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -137,11 +137,6 @@ public class FilterPartitionBenchmark JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java index 4d077d44388..ac6d568ec44 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java @@ -140,11 +140,6 @@ public class FilteredAggregatorBenchmark JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 3cb2f60b97d..87a421df5fc 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -158,11 +158,6 @@ public class GroupByTypeInterfaceBenchmark JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/InFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/InFilterBenchmark.java index 967e3fae0b7..7bdcaf1c6ab 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/InFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/InFilterBenchmark.java @@ -32,7 +32,7 @@ import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier; +import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -85,11 +85,6 @@ public class InFilterBenchmark final BitmapFactory bitmapFactory = new RoaringBitmapFactory(); final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance(); final Iterable ints = intGenerator(); - final GenericIndexed dictionary = GenericIndexed.fromIterable( - FluentIterable.from(ints) - .transform(Object::toString), - GenericIndexed.STRING_STRATEGY - ); final GenericIndexed dictionaryUtf8 = GenericIndexed.fromIterable( FluentIterable.from(ints) .transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))), @@ -109,7 +104,7 @@ public class InFilterBenchmark ); selector = new MockColumnIndexSelector( bitmapFactory, - new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null) + new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null) ); inFilter = new InDimFilter( "dummy", diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java index d2fed4b0fab..7ac9931da48 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -95,9 +95,6 @@ public class JoinAndLookupBenchmark @Param({"500000"}) int rows; - @Param({"0", "16384"}) - int columnCacheSizeBytes; - private File tmpDir = null; private QueryableIndex index = null; private Segment baseSegment = null; @@ -123,8 +120,7 @@ public class JoinAndLookupBenchmark public void setup() throws IOException { tmpDir = FileUtils.createTempDir(); - ColumnConfig columnConfig = () -> columnCacheSizeBytes; - index = JoinTestHelper.createFactIndexBuilder(columnConfig, tmpDir, rows).buildMMappedIndex(); + index = JoinTestHelper.createFactIndexBuilder(ColumnConfig.DEFAULT, tmpDir, rows).buildMMappedIndex(); final String prefix = "c."; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/LikeFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/LikeFilterBenchmark.java index ee000f45de3..0bbfadf50a2 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/LikeFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/LikeFilterBenchmark.java @@ -37,7 +37,7 @@ import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier; +import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -122,11 +122,6 @@ public class LikeFilterBenchmark final BitmapFactory bitmapFactory = new RoaringBitmapFactory(); final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance(); final List ints = generateInts(); - final GenericIndexed dictionary = GenericIndexed.fromIterable( - FluentIterable.from(ints) - .transform(Object::toString), - GenericIndexed.STRING_STRATEGY - ); final GenericIndexed dictionaryUtf8 = GenericIndexed.fromIterable( FluentIterable.from(ints) .transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))), @@ -145,7 +140,7 @@ public class LikeFilterBenchmark ); selector = new MockColumnIndexSelector( bitmapFactory, - new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null) + new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null) ); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java index b96716aaf6e..df66a36a553 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -138,11 +138,6 @@ public class TopNTypeInterfaceBenchmark JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/ColumnarLongsEncodeDataFromSegmentBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/ColumnarLongsEncodeDataFromSegmentBenchmark.java index 1fc373f713a..5834f25b284 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/ColumnarLongsEncodeDataFromSegmentBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/ColumnarLongsEncodeDataFromSegmentBenchmark.java @@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.LongsColumn; import org.apache.druid.segment.column.ValueType; @@ -118,10 +119,7 @@ public class ColumnarLongsEncodeDataFromSegmentBenchmark extends BaseColumnarLon File dataFile = new File(dir, getColumnDataFileName(segmentName, columnName)); if (!dataFile.exists()) { - final IndexIO indexIO = new IndexIO( - new DefaultObjectMapper(), - () -> 0 - ); + final IndexIO indexIO = new IndexIO(new DefaultObjectMapper(), ColumnConfig.DEFAULT); try (final QueryableIndex index = indexIO.loadIndex(new File(segmentPath))) { final Set columnNames = new LinkedHashSet<>(); columnNames.add(ColumnHolder.TIME_COLUMN_NAME); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java index a394c1d6842..b9e43059c57 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java @@ -31,6 +31,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.generator.DataGenerator; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -105,10 +106,7 @@ public class IndexMergeBenchmark InjectableValues.Std injectableValues = new InjectableValues.Std(); injectableValues.addValue(ExprMacroTable.class, ExprMacroTable.nil()); JSON_MAPPER.setInjectableValues(injectableValues); - INDEX_IO = new IndexIO( - JSON_MAPPER, - () -> 0 - ); + INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT); } @Setup diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java index cd3dae8b26e..14d296fda61 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java @@ -30,6 +30,7 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.generator.DataGenerator; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -74,10 +75,7 @@ public class IndexPersistBenchmark static { NullHandling.initializeForTests(); JSON_MAPPER = new DefaultObjectMapper(); - INDEX_IO = new IndexIO( - JSON_MAPPER, - () -> 0 - ); + INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index 8fd6ca75e9f..08c51b9edf2 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -174,11 +174,6 @@ public class GroupByBenchmark ), new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 76c4f036f9b..b7bfe56ff02 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -64,6 +64,7 @@ import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.generator.DataGenerator; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -136,10 +137,7 @@ public class ScanBenchmark static { JSON_MAPPER = new DefaultObjectMapper(); - INDEX_IO = new IndexIO( - JSON_MAPPER, - () -> 0 - ); + INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java index 322a577e786..9a383f119cb 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java @@ -143,11 +143,6 @@ public class SearchBenchmark JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java index ee54b17ceaa..97d20ca68ae 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java @@ -136,11 +136,6 @@ public class TimeseriesBenchmark JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java index 843eae489ae..e69bd15a5f0 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java @@ -134,11 +134,6 @@ public class TopNBenchmark JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java index 002608cf8eb..6baf2478ada 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java @@ -160,11 +160,6 @@ public class TimeCompareBenchmark JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 9698b28e431..b741728aad8 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1478,7 +1478,6 @@ Processing properties set on the MiddleManager will be passed through to Peons. |`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s| |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| -|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| |`druid.processing.fifo`|Enables the processing queue to treat tasks of equal priority in a FIFO manner.|`true`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| |`druid.processing.intermediaryData.storage.type`|Storage type for intermediary segments of data shuffle between native parallel index tasks.
    Set to `local` to store segment files in the local storage of the MiddleManager or Indexer.
    Set to `deepstore` to use configured deep storage for better fault tolerance during rolling updates. When the storage type is `deepstore`, Druid stores the data in the `shuffle-data` directory under the configured deep storage path. Druid does not support automated cleanup for the `shuffle-data` directory. You can set up cloud storage lifecycle rules for automated cleanup of data at the `shuffle-data` prefix location.|`local`| @@ -1628,7 +1627,6 @@ Druid uses Jetty to serve HTTP requests. |`druid.processing.formatString`|Indexer processes use this format string to name their processing threads.|processing-%s| |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| -|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| @@ -1738,7 +1736,6 @@ Druid uses Jetty to serve HTTP requests. |`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s| |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| -|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| @@ -1917,7 +1914,6 @@ The broker uses processing configs for nested groupBy queries. |`druid.processing.buffer.poolCacheInitialCount`|initializes the number of buffers allocated on the intermediate results pool. Note that pool can create more buffers if necessary.|`0`| |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE| |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| -|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| |`druid.processing.merge.useParallelMergePool`|Enable automatic parallel merging for Brokers on a dedicated async ForkJoinPool. If `false`, instead merges will be done serially on the `HTTP` thread pool.|`true`| diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java index 9b148ac2548..b0dbacee242 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/WorkerChatHandlerTest.java @@ -34,6 +34,7 @@ import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; @@ -68,7 +69,7 @@ public class WorkerChatHandlerTest public void setUp() { ObjectMapper mapper = new DefaultObjectMapper(); - IndexIO indexIO = new IndexIO(mapper, () -> 0); + IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT); IndexMergerV9 indexMerger = new IndexMergerV9( mapper, indexIO, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index ee2f3b65b1b..39fa01b3da4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -61,6 +61,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusher; @@ -137,7 +138,7 @@ public class CalciteMSQTestsHelper ) ); ObjectMapper testMapper = MSQTestBase.setupObjectMapper(dummyInjector); - IndexIO indexIO = new IndexIO(testMapper, () -> 0); + IndexIO indexIO = new IndexIO(testMapper, ColumnConfig.DEFAULT); SegmentCacheManager segmentCacheManager = null; try { segmentCacheManager = new SegmentCacheManagerFactory(testMapper).manufacturate(temporaryFolder.newFolder( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 772279ada79..736ec2f430d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -122,6 +122,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.loading.DataSegmentPusher; @@ -356,7 +357,7 @@ public class MSQTestBase extends BaseCalciteQueryTest ); ObjectMapper secondMapper = setupObjectMapper(secondInjector); - indexIO = new IndexIO(secondMapper, () -> 0); + indexIO = new IndexIO(secondMapper, ColumnConfig.DEFAULT); try { segmentCacheManager = new SegmentCacheManagerFactory(secondMapper).manufacturate(tmpFolder.newFolder("test")); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index 655077008db..a6f98b3ba85 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -40,6 +40,7 @@ import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.NoopRowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; @@ -114,10 +115,7 @@ public class MSQTestWorkerContext implements WorkerContext @Override public FrameContext frameContext(QueryDefinition queryDef, int stageNumber) { - IndexIO indexIO = new IndexIO( - mapper, - () -> 0 - ); + IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT); IndexMergerV9 indexMerger = new IndexMergerV9( mapper, indexIO, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java index abb4273cd41..44d629b2276 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; @@ -82,10 +83,7 @@ public class TestUtils public TestUtils() { this.jsonMapper = new DefaultObjectMapper(); - indexIO = new IndexIO( - jsonMapper, - () -> 0 - ); + indexIO = new IndexIO(jsonMapper, ColumnConfig.DEFAULT); indexMergerV9Factory = new IndexMergerV9Factory( jsonMapper, indexIO, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java index 62258ceb48d..c0489c61b42 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java @@ -38,6 +38,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -179,10 +180,7 @@ public class AppenderatorsTest ); metrics = new FireDepartmentMetrics(); - IndexIO indexIO = new IndexIO( - objectMapper, - () -> 0 - ); + IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); IndexMergerV9 indexMerger = new IndexMergerV9( objectMapper, indexIO, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java index d0ba1de3ce1..f8100772235 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java @@ -41,6 +41,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -184,10 +185,7 @@ public class BatchAppenderatorsTest ); metrics = new FireDepartmentMetrics(); - IndexIO indexIO = new IndexIO( - objectMapper, - () -> 0 - ); + IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); IndexMergerV9 indexMerger = new IndexMergerV9( objectMapper, indexIO, @@ -574,7 +572,7 @@ public class BatchAppenderatorsTest .config(config) .joinableFactory(NoopJoinableFactory.INSTANCE) .jsonMapper(mapper) - .indexIO(new IndexIO(new ObjectMapper(), () -> 0)) + .indexIO(new IndexIO(new ObjectMapper(), ColumnConfig.DEFAULT)) .indexMergerV9(indexMergerV9) .taskReportFileWriter(new NoopTestTaskReportFileWriter()) .authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index ad9841bf26a..2f6d53e1a9a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -104,6 +104,7 @@ import org.apache.druid.segment.SimpleQueryableIndex; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; @@ -1983,7 +1984,7 @@ public class CompactionTaskTest Map segmentFileMap ) { - super(mapper, () -> 0); + super(mapper, ColumnConfig.DEFAULT); queryableIndexMap = Maps.newHashMapWithExpectedSize(segmentFileMap.size()); for (Entry entry : segmentFileMap.entrySet()) { diff --git a/pom.xml b/pom.xml index 6d0d1337498..934a25b88be 100644 --- a/pom.xml +++ b/pom.xml @@ -236,7 +236,7 @@ false - + sigar @@ -246,7 +246,7 @@ - + ${repoOrgId} diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java index ac6c270bd3d..823c9e71efb 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java @@ -142,13 +142,6 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig implem return DEFAULT_NUM_MERGE_BUFFERS; } - @Override - @Config(value = "${base_path}.columnCache.sizeBytes") - public int columnCacheSizeBytes() - { - return 0; - } - @Override @Config(value = "${base_path}.indexes.skipValueRangeIndexScale") public double skipValueRangeIndexScale() diff --git a/processing/src/main/java/org/apache/druid/segment/IndexIO.java b/processing/src/main/java/org/apache/druid/segment/IndexIO.java index c88edbb122d..f2d57a5517e 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexIO.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexIO.java @@ -65,10 +65,10 @@ import org.apache.druid.segment.data.IndexedIterable; import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.data.VSizeColumnarMultiInts; import org.apache.druid.segment.serde.ComplexColumnPartSupplier; -import org.apache.druid.segment.serde.DictionaryEncodedColumnSupplier; -import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier; import org.apache.druid.segment.serde.FloatNumericColumnSupplier; import org.apache.druid.segment.serde.LongNumericColumnSupplier; +import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier; +import org.apache.druid.segment.serde.StringUtf8DictionaryEncodedColumnSupplier; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -109,7 +109,7 @@ public class IndexIO this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); Preconditions.checkNotNull(columnConfig, "null ColumnConfig"); ImmutableMap.Builder indexLoadersBuilder = ImmutableMap.builder(); - LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler(), columnConfig); + LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler()); for (int i = 0; i <= V8_VERSION; i++) { indexLoadersBuilder.put(i, legacyIndexLoader); } @@ -364,7 +364,6 @@ public class IndexIO metrics.put(metric, holder); } - Map> dimValueLookups = new HashMap<>(); Map> dimValueUtf8Lookups = new HashMap<>(); Map dimColumns = new HashMap<>(); Map> bitmaps = new HashMap<>(); @@ -379,8 +378,6 @@ public class IndexIO fileDimensionName ); - // Duplicate the first buffer since we are reading the dictionary twice. - dimValueLookups.put(dimension, GenericIndexed.read(dimBuffer.duplicate(), GenericIndexed.STRING_STRATEGY)); dimValueUtf8Lookups.put(dimension, GenericIndexed.read(dimBuffer, GenericIndexed.UTF8_STRATEGY)); dimColumns.put(dimension, VSizeColumnarMultiInts.readFromByteBuffer(dimBuffer)); } @@ -410,7 +407,6 @@ public class IndexIO dataInterval, timestamps, metrics, - dimValueLookups, dimValueUtf8Lookups, dimColumns, bitmaps, @@ -432,12 +428,10 @@ public class IndexIO static class LegacyIndexLoader implements IndexLoader { private final IndexIOHandler legacyHandler; - private final ColumnConfig columnConfig; - LegacyIndexLoader(IndexIOHandler legacyHandler, ColumnConfig columnConfig) + LegacyIndexLoader(IndexIOHandler legacyHandler) { this.legacyHandler = legacyHandler; - this.columnConfig = columnConfig; } @Override @@ -452,21 +446,18 @@ public class IndexIO .setType(ValueType.STRING) .setHasMultipleValues(true) .setDictionaryEncodedColumnSupplier( - new DictionaryEncodedColumnSupplier( - index.getDimValueLookup(dimension), - index.getDimValueUtf8Lookup(dimension), + new StringUtf8DictionaryEncodedColumnSupplier<>( + index.getDimValueUtf8Lookup(dimension)::singleThreaded, null, - Suppliers.ofInstance(index.getDimColumn(dimension)), - columnConfig.columnCacheSizeBytes() + Suppliers.ofInstance(index.getDimColumn(dimension)) ) ); GenericIndexed bitmaps = index.getBitmapIndexes().get(dimension); ImmutableRTree spatialIndex = index.getSpatialIndexes().get(dimension); builder.setIndexSupplier( - new DictionaryEncodedStringIndexSupplier( + new StringUtf8ColumnIndexSupplier<>( new ConciseBitmapFactory(), - index.getDimValueLookup(dimension), - index.getDimValueUtf8Lookup(dimension), + index.getDimValueUtf8Lookup(dimension)::singleThreaded, bitmaps, spatialIndex ), diff --git a/processing/src/main/java/org/apache/druid/segment/MMappedIndex.java b/processing/src/main/java/org/apache/druid/segment/MMappedIndex.java index 737a2be9f20..e0fdcebd809 100644 --- a/processing/src/main/java/org/apache/druid/segment/MMappedIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/MMappedIndex.java @@ -42,7 +42,6 @@ public class MMappedIndex final Interval dataInterval; final CompressedColumnarLongsSupplier timestamps; final Map metrics; - final Map> dimValueLookups; final Map> dimValueUtf8Lookups; final Map dimColumns; final Map> invertedIndexes; @@ -55,7 +54,6 @@ public class MMappedIndex Interval dataInterval, CompressedColumnarLongsSupplier timestamps, Map metrics, - Map> dimValueLookups, Map> dimValueUtf8Lookups, Map dimColumns, Map> invertedIndexes, @@ -68,7 +66,6 @@ public class MMappedIndex this.dataInterval = dataInterval; this.timestamps = timestamps; this.metrics = metrics; - this.dimValueLookups = dimValueLookups; this.dimValueUtf8Lookups = dimValueUtf8Lookups; this.dimColumns = dimColumns; this.invertedIndexes = invertedIndexes; @@ -97,11 +94,6 @@ public class MMappedIndex return metrics.get(metric); } - public GenericIndexed getDimValueLookup(String dimension) - { - return dimValueLookups.get(dimension); - } - public GenericIndexed getDimValueUtf8Lookup(String dimension) { return dimValueUtf8Lookups.get(dimension); diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java index 273f0dfb765..bad9dc6a6f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java @@ -21,7 +21,23 @@ package org.apache.druid.segment.column; public interface ColumnConfig { - int columnCacheSizeBytes(); + ColumnConfig DEFAULT = new ColumnConfig() {}; + + ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig() + { + + @Override + public double skipValueRangeIndexScale() + { + return 1.0; + } + + @Override + public double skipValuePredicateIndexScale() + { + return 1.0; + } + }; /** * If the total number of rows in a column multiplied by this value is smaller than the total number of bitmap diff --git a/processing/src/main/java/org/apache/druid/segment/column/IndexedStringDruidPredicateIndex.java b/processing/src/main/java/org/apache/druid/segment/column/IndexedStringDruidPredicateIndex.java index 49badda066c..9c5aa9af121 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/IndexedStringDruidPredicateIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/column/IndexedStringDruidPredicateIndex.java @@ -31,41 +31,12 @@ import java.util.NoSuchElementException; public final class IndexedStringDruidPredicateIndex> implements DruidPredicateIndex { - static final ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 0; - } - - @Override - public double skipValueRangeIndexScale() - { - return 1.0; - } - - @Override - public double skipValuePredicateIndexScale() - { - return 1.0; - } - }; private final BitmapFactory bitmapFactory; private final TDictionary dictionary; private final Indexed bitmaps; private final ColumnConfig columnConfig; private final int numRows; - public IndexedStringDruidPredicateIndex( - BitmapFactory bitmapFactory, - TDictionary dictionary, - Indexed bitmaps - ) - { - this(bitmapFactory, dictionary, bitmaps, ALWAYS_USE_INDEXES, Integer.MAX_VALUE); - } - public IndexedStringDruidPredicateIndex( BitmapFactory bitmapFactory, TDictionary dictionary, diff --git a/processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8LexicographicalRangeIndex.java b/processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8LexicographicalRangeIndex.java index 3f5c121e9ed..5ddb01ba02c 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8LexicographicalRangeIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8LexicographicalRangeIndex.java @@ -47,23 +47,6 @@ public final class IndexedUtf8LexicographicalRangeIndex bitmaps, - boolean hasNull - ) - { - this( - bitmapFactory, - dictionary, - bitmaps, - hasNull, - IndexedStringDruidPredicateIndex.ALWAYS_USE_INDEXES, - Integer.MAX_VALUE - ); - } - public IndexedUtf8LexicographicalRangeIndex( BitmapFactory bitmapFactory, TDictionary dictionary, diff --git a/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java deleted file mode 100644 index 69b3af4c140..00000000000 --- a/processing/src/main/java/org/apache/druid/segment/column/StringDictionaryEncodedColumn.java +++ /dev/null @@ -1,807 +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.druid.segment.column; - -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import org.apache.druid.query.extraction.ExtractionFn; -import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.AbstractDimensionSelector; -import org.apache.druid.segment.DimensionSelectorUtils; -import org.apache.druid.segment.IdLookup; -import org.apache.druid.segment.data.ColumnarInts; -import org.apache.druid.segment.data.ColumnarMultiInts; -import org.apache.druid.segment.data.Indexed; -import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.data.ReadableOffset; -import org.apache.druid.segment.data.SingleIndexedInt; -import org.apache.druid.segment.filter.BooleanValueMatcher; -import org.apache.druid.segment.historical.HistoricalDimensionSelector; -import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector; -import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; -import org.apache.druid.segment.vector.ReadableVectorInspector; -import org.apache.druid.segment.vector.ReadableVectorOffset; -import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; -import org.apache.druid.segment.vector.VectorObjectSelector; -import org.apache.druid.utils.CloseableUtils; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.BitSet; -import java.util.List; - -/** - * - */ -public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn -{ - @Nullable - private final ColumnarInts column; - @Nullable - private final ColumnarMultiInts multiValueColumn; - private final Indexed dictionary; - private final Indexed dictionaryUtf8; - - public StringDictionaryEncodedColumn( - @Nullable ColumnarInts singleValueColumn, - @Nullable ColumnarMultiInts multiValueColumn, - Indexed dictionary, - Indexed dictionaryUtf8 - ) - { - this.column = singleValueColumn; - this.multiValueColumn = multiValueColumn; - this.dictionary = dictionary; - this.dictionaryUtf8 = dictionaryUtf8; - } - - @Override - public int length() - { - return hasMultipleValues() ? multiValueColumn.size() : column.size(); - } - - @Override - public boolean hasMultipleValues() - { - return column == null; - } - - @Override - public int getSingleValueRow(int rowNum) - { - return column.get(rowNum); - } - - @Override - public IndexedInts getMultiValueRow(int rowNum) - { - return multiValueColumn.get(rowNum); - } - - @Override - @Nullable - public String lookupName(int id) - { - return dictionary.get(id); - } - - - /** - * Returns the value for a particular dictionary id as UTF-8 bytes. - * - * The returned buffer is in big-endian order. It is not reused, so callers may modify the position, limit, byte - * order, etc of the buffer. - * - * The returned buffer points to the original data, so callers must take care not to use it outside the valid - * lifetime of this column. - * - * @param id id to lookup the dictionary value for - * - * @return dictionary value for the given id, or null if the value is itself null - */ - @Nullable - public ByteBuffer lookupNameUtf8(int id) - { - return dictionaryUtf8.get(id); - } - - @Override - public int lookupId(String name) - { - return dictionary.indexOf(name); - } - - @Override - public int getCardinality() - { - return dictionary.size(); - } - - @Override - public HistoricalDimensionSelector makeDimensionSelector( - final ReadableOffset offset, - @Nullable final ExtractionFn extractionFn - ) - { - abstract class QueryableDimensionSelector extends AbstractDimensionSelector - implements HistoricalDimensionSelector, IdLookup - { - @Override - public int getValueCardinality() - { - /* - This is technically wrong if - extractionFn != null && (extractionFn.getExtractionType() != ExtractionFn.ExtractionType.ONE_TO_ONE || - !extractionFn.preservesOrdering()) - However current behavior allows some GroupBy-V1 queries to work that wouldn't work otherwise and doesn't - cause any problems due to special handling of extractionFn everywhere. - See https://github.com/apache/druid/pull/8433 - */ - return getCardinality(); - } - - @Override - public String lookupName(int id) - { - final String value = StringDictionaryEncodedColumn.this.lookupName(id); - return extractionFn == null ? value : extractionFn.apply(value); - } - - @Nullable - @Override - public ByteBuffer lookupNameUtf8(int id) - { - return StringDictionaryEncodedColumn.this.lookupNameUtf8(id); - } - - @Override - public boolean supportsLookupNameUtf8() - { - return true; - } - - @Override - public boolean nameLookupPossibleInAdvance() - { - return true; - } - - @Nullable - @Override - public IdLookup idLookup() - { - return extractionFn == null ? this : null; - } - - @Override - public int lookupId(String name) - { - if (extractionFn != null) { - throw new UnsupportedOperationException("cannot perform lookup when applying an extraction function"); - } - return StringDictionaryEncodedColumn.this.lookupId(name); - } - } - - if (hasMultipleValues()) { - class MultiValueDimensionSelector extends QueryableDimensionSelector - { - @Override - public IndexedInts getRow() - { - return multiValueColumn.get(offset.getOffset()); - } - - @Override - public IndexedInts getRow(int offset) - { - return multiValueColumn.get(offset); - } - - @Override - public ValueMatcher makeValueMatcher(@Nullable String value) - { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, value); - } - - @Override - public ValueMatcher makeValueMatcher(Predicate predicate) - { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); - } - - @Nullable - @Override - public Object getObject() - { - return defaultGetObject(); - } - - @Override - public Class classOfObject() - { - return Object.class; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("multiValueColumn", multiValueColumn); - inspector.visit("offset", offset); - inspector.visit("extractionFn", extractionFn); - } - } - return new MultiValueDimensionSelector(); - } else { - class SingleValueQueryableDimensionSelector extends QueryableDimensionSelector - implements SingleValueHistoricalDimensionSelector - { - private final SingleIndexedInt row = new SingleIndexedInt(); - - @Override - public IndexedInts getRow() - { - row.setValue(getRowValue()); - return row; - } - - public int getRowValue() - { - return column.get(offset.getOffset()); - } - - @Override - public IndexedInts getRow(int offset) - { - row.setValue(getRowValue(offset)); - return row; - } - - @Override - public int getRowValue(int offset) - { - return column.get(offset); - } - - @Override - public ValueMatcher makeValueMatcher(final @Nullable String value) - { - if (extractionFn == null) { - final int valueId = lookupId(value); - if (valueId >= 0) { - return new ValueMatcher() - { - @Override - public boolean matches() - { - return getRowValue() == valueId; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("column", StringDictionaryEncodedColumn.this); - } - }; - } else { - return BooleanValueMatcher.of(false); - } - } else { - // Employ caching BitSet optimization - return makeValueMatcher(Predicates.equalTo(value)); - } - } - - @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) - { - final BitSet checkedIds = new BitSet(getCardinality()); - final BitSet matchingIds = new BitSet(getCardinality()); - - // Lazy matcher; only check an id if matches() is called. - return new ValueMatcher() - { - @Override - public boolean matches() - { - final int id = getRowValue(); - - if (checkedIds.get(id)) { - return matchingIds.get(id); - } else { - final boolean matches = predicate.apply(lookupName(id)); - checkedIds.set(id); - if (matches) { - matchingIds.set(id); - } - return matches; - } - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("column", StringDictionaryEncodedColumn.this); - } - }; - } - - @Override - public Object getObject() - { - return lookupName(getRowValue()); - } - - @Override - public Class classOfObject() - { - return String.class; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("column", column); - inspector.visit("offset", offset); - inspector.visit("extractionFn", extractionFn); - } - } - return new SingleValueQueryableDimensionSelector(); - } - } - - @Override - public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(final ReadableVectorOffset offset) - { - final class StringVectorSelector extends StringSingleValueDimensionVectorSelector - { - public StringVectorSelector() - { - super(column, offset); - } - - @Override - public int getValueCardinality() - { - return getCardinality(); - } - - @Nullable - @Override - public String lookupName(final int id) - { - return StringDictionaryEncodedColumn.this.lookupName(id); - } - - @Nullable - @Override - public ByteBuffer lookupNameUtf8(int id) - { - return StringDictionaryEncodedColumn.this.lookupNameUtf8(id); - } - - @Override - public int lookupId(@Nullable final String name) - { - return StringDictionaryEncodedColumn.this.lookupId(name); - } - } - - return new StringVectorSelector(); - } - - @Override - public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(final ReadableVectorOffset offset) - { - final class MultiStringVectorSelector extends StringMultiValueDimensionVectorSelector - { - - public MultiStringVectorSelector() - { - super(multiValueColumn, offset); - } - - @Override - public int getValueCardinality() - { - return getCardinality(); - } - - @Nullable - @Override - public String lookupName(final int id) - { - return StringDictionaryEncodedColumn.this.lookupName(id); - } - - @Nullable - @Override - public ByteBuffer lookupNameUtf8(int id) - { - return StringDictionaryEncodedColumn.this.lookupNameUtf8(id); - } - - @Override - public int lookupId(@Nullable final String name) - { - return StringDictionaryEncodedColumn.this.lookupId(name); - } - } - return new MultiStringVectorSelector(); - } - - @Override - public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset) - { - if (!hasMultipleValues()) { - final class StringVectorSelector extends StringVectorObjectSelector - { - public StringVectorSelector() - { - super(column, offset); - } - - @Nullable - @Override - public String lookupName(int id) - { - return StringDictionaryEncodedColumn.this.lookupName(id); - } - } - - return new StringVectorSelector(); - } else { - final class MultiStringVectorSelector extends MultiValueStringVectorObjectSelector - { - public MultiStringVectorSelector() - { - super(multiValueColumn, offset); - } - - @Nullable - @Override - public String lookupName(int id) - { - return StringDictionaryEncodedColumn.this.lookupName(id); - } - } - - return new MultiStringVectorSelector(); - } - } - - @Override - public void close() throws IOException - { - CloseableUtils.closeAll( - dictionary instanceof Closeable ? (Closeable) dictionary : null /* Dictionary may be CachingIndexed */, - column, - multiValueColumn - ); - } - - /** - * Base type for a {@link SingleValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING} - * built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods. - * - * Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}. - */ - public abstract static class StringSingleValueDimensionVectorSelector - implements SingleValueDimensionVectorSelector, IdLookup - { - private final ColumnarInts column; - private final ReadableVectorOffset offset; - private final int[] vector; - private int id = ReadableVectorInspector.NULL_ID; - - public StringSingleValueDimensionVectorSelector( - ColumnarInts column, - ReadableVectorOffset offset - ) - { - this.column = column; - this.offset = offset; - this.vector = new int[offset.getMaxVectorSize()]; - } - - @Override - public int[] getRowVector() - { - if (id == offset.getId()) { - return vector; - } - - if (offset.isContiguous()) { - column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize()); - } else { - column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize()); - } - - id = offset.getId(); - return vector; - } - - @Override - public boolean supportsLookupNameUtf8() - { - return true; - } - - @Override - public boolean nameLookupPossibleInAdvance() - { - return true; - } - - @Nullable - @Override - public IdLookup idLookup() - { - return this; - } - - @Override - public int getCurrentVectorSize() - { - return offset.getCurrentVectorSize(); - } - - @Override - public int getMaxVectorSize() - { - return offset.getMaxVectorSize(); - } - } - - /** - * Base type for a {@link MultiValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING} - * built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods. - * - * Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}. - */ - public abstract static class StringMultiValueDimensionVectorSelector - implements MultiValueDimensionVectorSelector, IdLookup - { - private final ColumnarMultiInts multiValueColumn; - private final ReadableVectorOffset offset; - - private final IndexedInts[] vector; - private int id = ReadableVectorInspector.NULL_ID; - - public StringMultiValueDimensionVectorSelector( - ColumnarMultiInts multiValueColumn, - ReadableVectorOffset offset - ) - { - this.multiValueColumn = multiValueColumn; - this.offset = offset; - this.vector = new IndexedInts[offset.getMaxVectorSize()]; - } - - @Override - public IndexedInts[] getRowVector() - { - if (id == offset.getId()) { - return vector; - } - - if (offset.isContiguous()) { - final int currentOffset = offset.getStartOffset(); - final int numRows = offset.getCurrentVectorSize(); - - for (int i = 0; i < numRows; i++) { - // Must use getUnshared, otherwise all elements in the vector could be the same shared object. - vector[i] = multiValueColumn.getUnshared(i + currentOffset); - } - } else { - final int[] offsets = offset.getOffsets(); - final int numRows = offset.getCurrentVectorSize(); - - for (int i = 0; i < numRows; i++) { - // Must use getUnshared, otherwise all elements in the vector could be the same shared object. - vector[i] = multiValueColumn.getUnshared(offsets[i]); - } - } - - id = offset.getId(); - return vector; - } - - @Override - public boolean supportsLookupNameUtf8() - { - return true; - } - - @Override - public boolean nameLookupPossibleInAdvance() - { - return true; - } - - @Nullable - @Override - public IdLookup idLookup() - { - return this; - } - @Override - public int getCurrentVectorSize() - { - return offset.getCurrentVectorSize(); - } - - @Override - public int getMaxVectorSize() - { - return offset.getMaxVectorSize(); - } - } - - /** - * Base type for a {@link VectorObjectSelector} for a dictionary encoded {@link ColumnType#STRING} - * built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods. - */ - public abstract static class StringVectorObjectSelector implements VectorObjectSelector - { - private final ColumnarInts column; - private final ReadableVectorOffset offset; - - private final int[] vector; - private final Object[] strings; - private int id = ReadableVectorInspector.NULL_ID; - - public StringVectorObjectSelector( - ColumnarInts column, - ReadableVectorOffset offset - ) - { - this.column = column; - this.offset = offset; - this.vector = new int[offset.getMaxVectorSize()]; - this.strings = new Object[offset.getMaxVectorSize()]; - } - - @Override - public Object[] getObjectVector() - { - if (id == offset.getId()) { - return strings; - } - - if (offset.isContiguous()) { - column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize()); - } else { - column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize()); - } - for (int i = 0; i < offset.getCurrentVectorSize(); i++) { - strings[i] = lookupName(vector[i]); - } - id = offset.getId(); - - return strings; - } - - @Override - public int getMaxVectorSize() - { - return offset.getMaxVectorSize(); - } - - @Override - public int getCurrentVectorSize() - { - return offset.getCurrentVectorSize(); - } - - @Nullable - public abstract String lookupName(int id); - } - - /** - * Base type for a {@link VectorObjectSelector} for a dictionary encoded {@link ColumnType#STRING} - * built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods. - */ - public abstract static class MultiValueStringVectorObjectSelector implements VectorObjectSelector - { - private final ColumnarMultiInts multiValueColumn; - private final ReadableVectorOffset offset; - - private final IndexedInts[] vector; - private final Object[] strings; - private int id = ReadableVectorInspector.NULL_ID; - - public MultiValueStringVectorObjectSelector( - ColumnarMultiInts multiValueColumn, - ReadableVectorOffset offset - ) - { - this.multiValueColumn = multiValueColumn; - this.offset = offset; - this.vector = new IndexedInts[offset.getMaxVectorSize()]; - this.strings = new Object[offset.getMaxVectorSize()]; - } - - @Nullable - public abstract String lookupName(int id); - - @Override - public Object[] getObjectVector() - { - if (id == offset.getId()) { - return strings; - } - - if (offset.isContiguous()) { - final int currentOffset = offset.getStartOffset(); - final int numRows = offset.getCurrentVectorSize(); - - for (int i = 0; i < numRows; i++) { - // Must use getUnshared, otherwise all elements in the vector could be the same shared object. - vector[i] = multiValueColumn.getUnshared(i + currentOffset); - } - } else { - final int[] offsets = offset.getOffsets(); - final int numRows = offset.getCurrentVectorSize(); - - for (int i = 0; i < numRows; i++) { - // Must use getUnshared, otherwise all elements in the vector could be the same shared object. - vector[i] = multiValueColumn.getUnshared(offsets[i]); - } - } - - for (int i = 0; i < offset.getCurrentVectorSize(); i++) { - IndexedInts ithRow = vector[i]; - if (ithRow.size() == 0) { - strings[i] = null; - } else if (ithRow.size() == 1) { - strings[i] = lookupName(ithRow.get(0)); - } else { - List row = new ArrayList<>(ithRow.size()); - // noinspection SSBasedInspection - for (int j = 0; j < ithRow.size(); j++) { - row.add(lookupName(ithRow.get(j))); - } - strings[i] = row; - } - } - - id = offset.getId(); - return strings; - } - - @Override - public int getMaxVectorSize() - { - return offset.getMaxVectorSize(); - } - - @Override - public int getCurrentVectorSize() - { - return offset.getCurrentVectorSize(); - } - } -} diff --git a/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java index baf7a4be9c7..63f2b36313c 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java @@ -39,6 +39,7 @@ import org.apache.druid.segment.historical.HistoricalDimensionSelector; import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector; import org.apache.druid.segment.nested.NestedCommonFormatColumn; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; +import org.apache.druid.segment.vector.ReadableVectorInspector; import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorObjectSelector; @@ -47,19 +48,18 @@ import org.apache.druid.utils.CloseableUtils; import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.BitSet; +import java.util.List; /** - * {@link DictionaryEncodedColumn} for a column which has only a UTF-8 dictionary, no String dictionary. + * {@link DictionaryEncodedColumn} for a column which has a {@link ByteBuffer} based UTF-8 dictionary. *

    - * This class is otherwise nearly identical to {@link StringDictionaryEncodedColumn} other than lacking a - * String dictionary. *

    * Implements {@link NestedCommonFormatColumn} so it can be used as a reader for single value string specializations * of {@link org.apache.druid.segment.AutoTypeColumnIndexer}. */ -public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColumn, - NestedCommonFormatColumn +public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColumn, NestedCommonFormatColumn { @Nullable private final ColumnarInts column; @@ -364,7 +364,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum @Override public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(final ReadableVectorOffset offset) { - final class StringVectorSelector extends StringDictionaryEncodedColumn.StringSingleValueDimensionVectorSelector + final class StringVectorSelector extends StringSingleValueDimensionVectorSelector { public StringVectorSelector() { @@ -404,7 +404,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum @Override public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(final ReadableVectorOffset offset) { - final class MultiStringVectorSelector extends StringDictionaryEncodedColumn.StringMultiValueDimensionVectorSelector + final class MultiStringVectorSelector extends StringMultiValueDimensionVectorSelector { public MultiStringVectorSelector() { @@ -446,7 +446,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset) { if (!hasMultipleValues()) { - final class StringVectorSelector extends StringDictionaryEncodedColumn.StringVectorObjectSelector + final class StringVectorSelector extends StringVectorObjectSelector { public StringVectorSelector() { @@ -462,7 +462,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum } return new StringVectorSelector(); } else { - final class MultiStringVectorSelector extends StringDictionaryEncodedColumn.MultiValueStringVectorObjectSelector + final class MultiStringVectorSelector extends MultiValueStringVectorObjectSelector { public MultiStringVectorSelector() { @@ -497,4 +497,309 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum { return new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary); } + + + + /** + * Base type for a {@link SingleValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING} + * built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods. + * + * Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}. + */ + public abstract static class StringSingleValueDimensionVectorSelector + implements SingleValueDimensionVectorSelector, IdLookup + { + private final ColumnarInts column; + private final ReadableVectorOffset offset; + private final int[] vector; + private int id = ReadableVectorInspector.NULL_ID; + + public StringSingleValueDimensionVectorSelector( + ColumnarInts column, + ReadableVectorOffset offset + ) + { + this.column = column; + this.offset = offset; + this.vector = new int[offset.getMaxVectorSize()]; + } + + @Override + public int[] getRowVector() + { + if (id == offset.getId()) { + return vector; + } + + if (offset.isContiguous()) { + column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize()); + } else { + column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize()); + } + + id = offset.getId(); + return vector; + } + + @Override + public boolean supportsLookupNameUtf8() + { + return true; + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + return true; + } + + @Nullable + @Override + public IdLookup idLookup() + { + return this; + } + + @Override + public int getCurrentVectorSize() + { + return offset.getCurrentVectorSize(); + } + + @Override + public int getMaxVectorSize() + { + return offset.getMaxVectorSize(); + } + } + + /** + * Base type for a {@link MultiValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING} + * built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods. + * + * Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}. + */ + public abstract static class StringMultiValueDimensionVectorSelector + implements MultiValueDimensionVectorSelector, IdLookup + { + private final ColumnarMultiInts multiValueColumn; + private final ReadableVectorOffset offset; + + private final IndexedInts[] vector; + private int id = ReadableVectorInspector.NULL_ID; + + public StringMultiValueDimensionVectorSelector( + ColumnarMultiInts multiValueColumn, + ReadableVectorOffset offset + ) + { + this.multiValueColumn = multiValueColumn; + this.offset = offset; + this.vector = new IndexedInts[offset.getMaxVectorSize()]; + } + + @Override + public IndexedInts[] getRowVector() + { + if (id == offset.getId()) { + return vector; + } + + if (offset.isContiguous()) { + final int currentOffset = offset.getStartOffset(); + final int numRows = offset.getCurrentVectorSize(); + + for (int i = 0; i < numRows; i++) { + // Must use getUnshared, otherwise all elements in the vector could be the same shared object. + vector[i] = multiValueColumn.getUnshared(i + currentOffset); + } + } else { + final int[] offsets = offset.getOffsets(); + final int numRows = offset.getCurrentVectorSize(); + + for (int i = 0; i < numRows; i++) { + // Must use getUnshared, otherwise all elements in the vector could be the same shared object. + vector[i] = multiValueColumn.getUnshared(offsets[i]); + } + } + + id = offset.getId(); + return vector; + } + + @Override + public boolean supportsLookupNameUtf8() + { + return true; + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + return true; + } + + @Nullable + @Override + public IdLookup idLookup() + { + return this; + } + @Override + public int getCurrentVectorSize() + { + return offset.getCurrentVectorSize(); + } + + @Override + public int getMaxVectorSize() + { + return offset.getMaxVectorSize(); + } + } + + /** + * Base type for a {@link VectorObjectSelector} for a dictionary encoded {@link ColumnType#STRING} + * built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods. + */ + public abstract static class StringVectorObjectSelector implements VectorObjectSelector + { + private final ColumnarInts column; + private final ReadableVectorOffset offset; + + private final int[] vector; + private final Object[] strings; + private int id = ReadableVectorInspector.NULL_ID; + + public StringVectorObjectSelector( + ColumnarInts column, + ReadableVectorOffset offset + ) + { + this.column = column; + this.offset = offset; + this.vector = new int[offset.getMaxVectorSize()]; + this.strings = new Object[offset.getMaxVectorSize()]; + } + + @Override + public Object[] getObjectVector() + { + if (id == offset.getId()) { + return strings; + } + + if (offset.isContiguous()) { + column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize()); + } else { + column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize()); + } + for (int i = 0; i < offset.getCurrentVectorSize(); i++) { + strings[i] = lookupName(vector[i]); + } + id = offset.getId(); + + return strings; + } + + @Override + public int getMaxVectorSize() + { + return offset.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return offset.getCurrentVectorSize(); + } + + @Nullable + public abstract String lookupName(int id); + } + + /** + * Base type for a {@link VectorObjectSelector} for a dictionary encoded {@link ColumnType#STRING} + * built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods. + */ + public abstract static class MultiValueStringVectorObjectSelector implements VectorObjectSelector + { + private final ColumnarMultiInts multiValueColumn; + private final ReadableVectorOffset offset; + + private final IndexedInts[] vector; + private final Object[] strings; + private int id = ReadableVectorInspector.NULL_ID; + + public MultiValueStringVectorObjectSelector( + ColumnarMultiInts multiValueColumn, + ReadableVectorOffset offset + ) + { + this.multiValueColumn = multiValueColumn; + this.offset = offset; + this.vector = new IndexedInts[offset.getMaxVectorSize()]; + this.strings = new Object[offset.getMaxVectorSize()]; + } + + @Nullable + public abstract String lookupName(int id); + + @Override + public Object[] getObjectVector() + { + if (id == offset.getId()) { + return strings; + } + + if (offset.isContiguous()) { + final int currentOffset = offset.getStartOffset(); + final int numRows = offset.getCurrentVectorSize(); + + for (int i = 0; i < numRows; i++) { + // Must use getUnshared, otherwise all elements in the vector could be the same shared object. + vector[i] = multiValueColumn.getUnshared(i + currentOffset); + } + } else { + final int[] offsets = offset.getOffsets(); + final int numRows = offset.getCurrentVectorSize(); + + for (int i = 0; i < numRows; i++) { + // Must use getUnshared, otherwise all elements in the vector could be the same shared object. + vector[i] = multiValueColumn.getUnshared(offsets[i]); + } + } + + for (int i = 0; i < offset.getCurrentVectorSize(); i++) { + IndexedInts ithRow = vector[i]; + if (ithRow.size() == 0) { + strings[i] = null; + } else if (ithRow.size() == 1) { + strings[i] = lookupName(ithRow.get(0)); + } else { + List row = new ArrayList<>(ithRow.size()); + // noinspection SSBasedInspection + for (int j = 0; j < ithRow.size(); j++) { + row.add(lookupName(ithRow.get(j))); + } + strings[i] = row; + } + } + + id = offset.getId(); + return strings; + } + + @Override + public int getMaxVectorSize() + { + return offset.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return offset.getCurrentVectorSize(); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/CachingIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/CachingIndexed.java deleted file mode 100644 index 6632eaa95e6..00000000000 --- a/processing/src/main/java/org/apache/druid/segment/data/CachingIndexed.java +++ /dev/null @@ -1,158 +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.druid.segment.data; - -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; - -import javax.annotation.Nullable; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.function.ToIntFunction; - -public class CachingIndexed implements CloseableIndexed -{ - private static final int INITIAL_CACHE_CAPACITY = 16384; - - private static final Logger log = new Logger(CachingIndexed.class); - - private final Indexed delegate; - private final ToIntFunction sizeFn; - @Nullable - private final SizedLRUMap cachedValues; - - /** - * Creates a CachingIndexed wrapping the given GenericIndexed with a value lookup cache - * - * CachingIndexed objects are not thread safe and should only be used by a single thread at a time. - * CachingIndexed objects must be closed to release any underlying cache resources. - * - * @param delegate the Indexed to wrap with a lookup cache. - * @param sizeFn function that determines the size in bytes of an object - * @param lookupCacheSize maximum size in bytes of the lookup cache if greater than zero - */ - public CachingIndexed(Indexed delegate, final ToIntFunction sizeFn, final int lookupCacheSize) - { - this.delegate = delegate; - this.sizeFn = sizeFn; - - if (lookupCacheSize > 0) { - log.debug("Allocating column cache of max size[%d]", lookupCacheSize); - cachedValues = new SizedLRUMap<>(INITIAL_CACHE_CAPACITY, lookupCacheSize); - } else { - cachedValues = null; - } - } - - @Override - public int size() - { - return delegate.size(); - } - - @Override - public T get(int index) - { - if (cachedValues != null) { - final T cached = cachedValues.getValue(index); - if (cached != null) { - return cached; - } - - final T value = delegate.get(index); - cachedValues.put(index, value, sizeFn.applyAsInt(value)); - return value; - } else { - return delegate.get(index); - } - } - - @Override - public int indexOf(@Nullable T value) - { - return delegate.indexOf(value); - } - - @Override - public boolean isSorted() - { - return delegate.isSorted(); - } - - @Override - public Iterator iterator() - { - return delegate.iterator(); - } - - @Override - public void close() - { - if (cachedValues != null) { - log.debug("Closing column cache"); - cachedValues.clear(); - } - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("cachedValues", cachedValues != null); - inspector.visit("delegate", delegate); - } - - private static class SizedLRUMap extends LinkedHashMap> - { - private final int maxBytes; - private int numBytes = 0; - - SizedLRUMap(int initialCapacity, int maxBytes) - { - super(initialCapacity, 0.75f, true); - this.maxBytes = maxBytes; - } - - @Override - protected boolean removeEldestEntry(Map.Entry> eldest) - { - if (numBytes > maxBytes) { - numBytes -= eldest.getValue().lhs; - return true; - } - return false; - } - - public void put(K key, @Nullable V value, int size) - { - final int totalSize = size + 48; // add approximate object overhead - numBytes += totalSize; - super.put(key, new Pair<>(totalSize, value)); - } - - @Nullable - public V getValue(Object key) - { - final Pair sizeValuePair = super.get(key); - return sizeValuePair == null ? null : sizeValuePair.rhs; - } - } -} diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java index 6dd627d652f..b21760416fe 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java @@ -40,7 +40,7 @@ import org.apache.druid.segment.IdLookup; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.StringDictionaryEncodedColumn; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.ColumnarDoubles; @@ -678,7 +678,7 @@ public class NestedFieldDictionaryEncodedColumn stringDictionary; - final Supplier frontCodedStringDictionarySupplier; + final Supplier> dictionarySupplier; final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, @@ -92,17 +77,19 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier stringDictionary; - private final Supplier frontCodedStringDictionarySupplier; + private final Supplier> dictionarySupplier; private final Supplier encodedColumnSupplier; private final GenericIndexed valueIndexes; - private final ImmutableBitmap nullValueBitmap; - private final BitmapFactory bitmapFactory; - private final ColumnConfig columnConfig; - private final int numRows; + private final ColumnIndexSupplier stringIndexSupplier; private ScalarStringColumnAndIndexSupplier( - GenericIndexed stringDictionary, - Supplier frontCodedStringDictionarySupplier, + Supplier> dictionarySupplier, Supplier encodedColumnSupplier, GenericIndexed valueIndexes, BitmapSerdeFactory serdeFactory, @@ -176,27 +158,23 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier( + serdeFactory.getBitmapFactory(), + dictionarySupplier, + valueIndexes, + null, + columnConfig, + numRows + ); } @Override public NestedCommonFormatColumn get() { - if (frontCodedStringDictionarySupplier != null) { - return new StringUtf8DictionaryEncodedColumn( - encodedColumnSupplier.get(), - null, - frontCodedStringDictionarySupplier.get() - ); - } - return new StringUtf8DictionaryEncodedColumn(encodedColumnSupplier.get(), null, stringDictionary.singleThreaded()); + return new StringUtf8DictionaryEncodedColumn(encodedColumnSupplier.get(), null, dictionarySupplier.get()); } @Nullable @@ -204,44 +182,7 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier T as(Class clazz) { if (valueIndexes != null) { - final Indexed singleThreadedBitmaps = valueIndexes.singleThreaded(); - final Indexed utf8Dictionary = frontCodedStringDictionarySupplier == null - ? stringDictionary.singleThreaded() - : frontCodedStringDictionarySupplier.get(); - if (clazz.equals(NullValueIndex.class)) { - final BitmapColumnIndex nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap); - return (T) (NullValueIndex) () -> nullIndex; - } else if (clazz.equals(StringValueSetIndex.class)) { - return (T) new IndexedUtf8ValueSetIndex<>( - bitmapFactory, - utf8Dictionary, - singleThreadedBitmaps - ); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new IndexedStringDruidPredicateIndex<>( - bitmapFactory, - new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary), - singleThreadedBitmaps, - columnConfig, - numRows - ); - } else if (clazz.equals(LexicographicalRangeIndex.class)) { - return (T) new IndexedUtf8LexicographicalRangeIndex<>( - bitmapFactory, - utf8Dictionary, - singleThreadedBitmaps, - utf8Dictionary.get(0) == null, - columnConfig, - numRows - ); - } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) - || clazz.equals(DictionaryEncodedValueIndex.class)) { - return (T) new IndexedStringDictionaryEncodedStringValueIndex<>( - bitmapFactory, - new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary), - valueIndexes - ); - } + return stringIndexSupplier.as(clazz); } return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java index befe16c5511..bf09e9fe157 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java @@ -40,8 +40,8 @@ import org.apache.druid.segment.IdLookup; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ColumnTypeFactory; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.StringDictionaryEncodedColumn; import org.apache.druid.segment.column.StringEncodingStrategies; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; @@ -668,7 +668,7 @@ public class VariantColumn> @Override public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(ReadableVectorOffset offset) { - final class StringVectorSelector extends StringDictionaryEncodedColumn.StringSingleValueDimensionVectorSelector + final class StringVectorSelector extends StringUtf8DictionaryEncodedColumn.StringSingleValueDimensionVectorSelector { public StringVectorSelector() { diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index b1eef3307b6..20ca158d1c6 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -48,6 +48,7 @@ import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.GenericIndexedWriter; import org.apache.druid.segment.data.ImmutableRTreeObjectStrategy; +import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.V3CompressedVSizeColumnarMultiIntsSupplier; import org.apache.druid.segment.data.VSizeColumnarInts; import org.apache.druid.segment.data.VSizeColumnarMultiInts; @@ -313,16 +314,21 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde final int dictionaryStartPosition = buffer.position(); final byte dictionaryVersion = buffer.get(); + final Supplier> dictionarySupplier; if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) { final byte encodingId = buffer.get(); if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) { - readFrontCodedColumn(buffer, builder, rVersion, rFlags, hasMultipleValues); + dictionarySupplier = FrontCodedIndexed.read(buffer, byteOrder); } else if (encodingId == StringEncodingStrategy.UTF8_ID) { // this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but // this provides backwards compatibility should we switch at some point in the future to always // writing dictionaryVersion - readGenericIndexedColumn(buffer, builder, columnConfig, rVersion, rFlags, hasMultipleValues); + dictionarySupplier = GenericIndexed.read( + buffer, + GenericIndexed.UTF8_STRATEGY, + builder.getFileMapper() + )::singleThreaded; } else { throw new ISE("impossible, unknown encoding strategy id: %s", encodingId); } @@ -331,101 +337,13 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde // as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the // GenericIndexed version can be correctly read buffer.position(dictionaryStartPosition); - readGenericIndexedColumn(buffer, builder, columnConfig, rVersion, rFlags, hasMultipleValues); - } - } - - private void readGenericIndexedColumn( - ByteBuffer buffer, - ColumnBuilder builder, - ColumnConfig columnConfig, - VERSION rVersion, - int rFlags, - boolean hasMultipleValues - ) - { - // Duplicate the first buffer since we are reading the dictionary twice. - final GenericIndexed rDictionary = GenericIndexed.read( - buffer.duplicate(), - GenericIndexed.STRING_STRATEGY, - builder.getFileMapper() - ); - - final GenericIndexed rDictionaryUtf8 = GenericIndexed.read( - buffer, - GenericIndexed.UTF8_STRATEGY, - builder.getFileMapper() - ); - - final WritableSupplier rSingleValuedColumn; - final WritableSupplier rMultiValuedColumn; - - if (hasMultipleValues) { - rMultiValuedColumn = readMultiValuedColumn(rVersion, buffer, rFlags); - rSingleValuedColumn = null; - } else { - rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer); - rMultiValuedColumn = null; - } - - final String firstDictionaryEntry = rDictionary.get(0); - - DictionaryEncodedColumnSupplier dictionaryEncodedColumnSupplier = new DictionaryEncodedColumnSupplier( - rDictionary, - rDictionaryUtf8, - rSingleValuedColumn, - rMultiValuedColumn, - columnConfig.columnCacheSizeBytes() - ); - - builder.setHasMultipleValues(hasMultipleValues) - .setHasNulls(firstDictionaryEntry == null) - .setDictionaryEncodedColumnSupplier(dictionaryEncodedColumnSupplier); - - GenericIndexed rBitmaps = null; - ImmutableRTree rSpatialIndex = null; - if (!Feature.NO_BITMAP_INDEX.isSet(rFlags)) { - rBitmaps = GenericIndexed.read( + dictionarySupplier = GenericIndexed.read( buffer, - bitmapSerdeFactory.getObjectStrategy(), + GenericIndexed.UTF8_STRATEGY, builder.getFileMapper() - ); + )::singleThreaded; } - if (buffer.hasRemaining()) { - rSpatialIndex = new ImmutableRTreeObjectStrategy( - bitmapSerdeFactory.getBitmapFactory() - ).fromByteBufferWithSize(buffer); - } - - if (rBitmaps != null || rSpatialIndex != null) { - builder.setIndexSupplier( - new DictionaryEncodedStringIndexSupplier( - bitmapSerdeFactory.getBitmapFactory(), - rDictionary, - rDictionaryUtf8, - rBitmaps, - rSpatialIndex - ), - rBitmaps != null, - rSpatialIndex != null - ); - } - } - - private void readFrontCodedColumn( - ByteBuffer buffer, - ColumnBuilder builder, - VERSION rVersion, - int rFlags, - boolean hasMultipleValues - ) - { - final Supplier rUtf8Dictionary = FrontCodedIndexed.read( - buffer, - byteOrder - ); - final WritableSupplier rSingleValuedColumn; final WritableSupplier rMultiValuedColumn; @@ -437,17 +355,16 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde rMultiValuedColumn = null; } - final boolean hasNulls = rUtf8Dictionary.get().get(0) == null; + final boolean hasNulls = dictionarySupplier.get().get(0) == null; - StringFrontCodedDictionaryEncodedColumnSupplier dictionaryEncodedColumnSupplier = - new StringFrontCodedDictionaryEncodedColumnSupplier( - rUtf8Dictionary, - rSingleValuedColumn, - rMultiValuedColumn - ); + final StringUtf8DictionaryEncodedColumnSupplier supplier = new StringUtf8DictionaryEncodedColumnSupplier<>( + dictionarySupplier, + rSingleValuedColumn, + rMultiValuedColumn + ); builder.setHasMultipleValues(hasMultipleValues) .setHasNulls(hasNulls) - .setDictionaryEncodedColumnSupplier(dictionaryEncodedColumnSupplier); + .setDictionaryEncodedColumnSupplier(supplier); GenericIndexed rBitmaps = null; ImmutableRTree rSpatialIndex = null; @@ -467,9 +384,9 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde if (rBitmaps != null || rSpatialIndex != null) { builder.setIndexSupplier( - new StringFrontCodedColumnIndexSupplier( + new StringUtf8ColumnIndexSupplier( bitmapSerdeFactory.getBitmapFactory(), - rUtf8Dictionary, + dictionarySupplier, rBitmaps, rSpatialIndex ), diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java deleted file mode 100644 index a8148ca6a25..00000000000 --- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnSupplier.java +++ /dev/null @@ -1,100 +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.druid.segment.serde; - -import com.google.common.base.Supplier; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.StringDictionaryEncodedColumn; -import org.apache.druid.segment.data.CachingIndexed; -import org.apache.druid.segment.data.ColumnarInts; -import org.apache.druid.segment.data.ColumnarMultiInts; -import org.apache.druid.segment.data.GenericIndexed; -import org.apache.druid.segment.data.Indexed; - -import javax.annotation.Nullable; -import java.nio.ByteBuffer; - -/** - * - */ -public class DictionaryEncodedColumnSupplier implements Supplier> -{ - private final GenericIndexed dictionary; - private final GenericIndexed dictionaryUtf8; - private final @Nullable Supplier singleValuedColumn; - private final @Nullable Supplier multiValuedColumn; - private final int lookupCacheSize; - - public DictionaryEncodedColumnSupplier( - GenericIndexed dictionary, - GenericIndexed dictionaryUtf8, - @Nullable Supplier singleValuedColumn, - @Nullable Supplier multiValuedColumn, - int lookupCacheSize - ) - { - this.dictionary = dictionary; - this.dictionaryUtf8 = dictionaryUtf8; - this.singleValuedColumn = singleValuedColumn; - this.multiValuedColumn = multiValuedColumn; - this.lookupCacheSize = lookupCacheSize; - } - - @Override - public DictionaryEncodedColumn get() - { - final Indexed cacheWrappedDictionary; - final Indexed singleThreadedDictionaryUtf8 = dictionaryUtf8.singleThreaded(); - - if (lookupCacheSize > 0) { - cacheWrappedDictionary = new CachingIndexed<>( - dictionary.singleThreaded(), - s -> s == null ? 0 : s.length() * Character.BYTES, - lookupCacheSize - ); - } else { - cacheWrappedDictionary = dictionary.singleThreaded(); - } - - if (NullHandling.mustCombineNullAndEmptyInDictionary(singleThreadedDictionaryUtf8)) { - return new StringDictionaryEncodedColumn( - singleValuedColumn != null ? new CombineFirstTwoValuesColumnarInts(singleValuedColumn.get()) : null, - multiValuedColumn != null ? new CombineFirstTwoValuesColumnarMultiInts(multiValuedColumn.get()) : null, - CombineFirstTwoEntriesIndexed.returnNull(cacheWrappedDictionary), - CombineFirstTwoEntriesIndexed.returnNull(singleThreadedDictionaryUtf8) - ); - } else if (NullHandling.mustReplaceFirstValueWithNullInDictionary(singleThreadedDictionaryUtf8)) { - return new StringDictionaryEncodedColumn( - singleValuedColumn != null ? singleValuedColumn.get() : null, - multiValuedColumn != null ? multiValuedColumn.get() : null, - new ReplaceFirstValueWithNullIndexed<>(cacheWrappedDictionary), - new ReplaceFirstValueWithNullIndexed<>(singleThreadedDictionaryUtf8) - ); - } else { - return new StringDictionaryEncodedColumn( - singleValuedColumn != null ? singleValuedColumn.get() : null, - multiValuedColumn != null ? multiValuedColumn.get() : null, - cacheWrappedDictionary, - singleThreadedDictionaryUtf8 - ); - } - } -} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplier.java deleted file mode 100644 index d7dc25b9796..00000000000 --- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplier.java +++ /dev/null @@ -1,127 +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.druid.segment.serde; - -import org.apache.druid.collections.bitmap.BitmapFactory; -import org.apache.druid.collections.bitmap.ImmutableBitmap; -import org.apache.druid.collections.spatial.ImmutableRTree; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.segment.column.BitmapColumnIndex; -import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.DictionaryEncodedValueIndex; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.IndexedStringDictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.IndexedStringDruidPredicateIndex; -import org.apache.druid.segment.column.IndexedUtf8LexicographicalRangeIndex; -import org.apache.druid.segment.column.IndexedUtf8ValueSetIndex; -import org.apache.druid.segment.column.LexicographicalRangeIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; -import org.apache.druid.segment.column.SpatialIndex; -import org.apache.druid.segment.column.StringValueSetIndex; -import org.apache.druid.segment.column.Utf8ValueSetIndex; -import org.apache.druid.segment.data.GenericIndexed; -import org.apache.druid.segment.data.Indexed; - -import javax.annotation.Nullable; -import java.nio.ByteBuffer; - -public class DictionaryEncodedStringIndexSupplier implements ColumnIndexSupplier -{ - private final BitmapFactory bitmapFactory; - private final GenericIndexed dictionary; - private final GenericIndexed dictionaryUtf8; - @Nullable - private final GenericIndexed bitmaps; - @Nullable - private final ImmutableRTree indexedTree; - - public DictionaryEncodedStringIndexSupplier( - BitmapFactory bitmapFactory, - GenericIndexed dictionary, - GenericIndexed dictionaryUtf8, - @Nullable GenericIndexed bitmaps, - @Nullable ImmutableRTree indexedTree - ) - { - this.bitmapFactory = bitmapFactory; - this.dictionary = dictionary; - this.dictionaryUtf8 = dictionaryUtf8; - this.bitmaps = bitmaps; - this.indexedTree = indexedTree; - } - - @Nullable - @Override - @SuppressWarnings("unchecked") - public T as(Class clazz) - { - if (bitmaps != null) { - Indexed singleThreadedStrings = dictionary.singleThreaded(); - Indexed singleThreadedUtf8 = dictionaryUtf8.singleThreaded(); - Indexed singleThreadedBitmaps = bitmaps.singleThreaded(); - - if (NullHandling.mustCombineNullAndEmptyInDictionary(singleThreadedUtf8)) { - singleThreadedStrings = CombineFirstTwoEntriesIndexed.returnNull(singleThreadedStrings); - singleThreadedUtf8 = CombineFirstTwoEntriesIndexed.returnNull(singleThreadedUtf8); - singleThreadedBitmaps = CombineFirstTwoEntriesIndexed.unionBitmaps(bitmapFactory, singleThreadedBitmaps); - } else if (NullHandling.mustReplaceFirstValueWithNullInDictionary(singleThreadedUtf8)) { - singleThreadedStrings = new ReplaceFirstValueWithNullIndexed<>(singleThreadedStrings); - singleThreadedUtf8 = new ReplaceFirstValueWithNullIndexed<>(singleThreadedUtf8); - } - - if (clazz.equals(NullValueIndex.class)) { - final BitmapColumnIndex nullIndex; - final ByteBuffer firstValue = singleThreadedUtf8.get(0); - if (NullHandling.isNullOrEquivalent(firstValue)) { - nullIndex = new SimpleImmutableBitmapIndex(singleThreadedBitmaps.get(0)); - } else { - nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap()); - } - return (T) (NullValueIndex) () -> nullIndex; - } else if (clazz.equals(StringValueSetIndex.class)) { - return (T) new IndexedUtf8ValueSetIndex<>(bitmapFactory, singleThreadedUtf8, singleThreadedBitmaps); - } else if (clazz.equals(Utf8ValueSetIndex.class)) { - return (T) new IndexedUtf8ValueSetIndex<>(bitmapFactory, singleThreadedUtf8, singleThreadedBitmaps); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new IndexedStringDruidPredicateIndex<>(bitmapFactory, singleThreadedStrings, singleThreadedBitmaps); - } else if (clazz.equals(LexicographicalRangeIndex.class)) { - return (T) new IndexedUtf8LexicographicalRangeIndex<>( - bitmapFactory, - singleThreadedUtf8, - singleThreadedBitmaps, - NullHandling.isNullOrEquivalent(singleThreadedStrings.get(0)) - ); - } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) - || clazz.equals(DictionaryEncodedValueIndex.class)) { - return (T) new IndexedStringDictionaryEncodedStringValueIndex<>( - bitmapFactory, - singleThreadedStrings, - singleThreadedBitmaps - ); - } - } - if (indexedTree != null && clazz.equals(SpatialIndex.class)) { - return (T) (SpatialIndex) () -> indexedTree; - } - return null; - } -} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java index 42e83985690..9b107b1d78b 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java @@ -23,7 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerdeFactory; @@ -128,106 +130,18 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde public Deserializer getDeserializer() { if (isVariantType || logicalType.isArray()) { - return ((buffer, builder, columnConfig) -> { - VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read( - logicalType, - byteOrder, - bitmapSerdeFactory, - buffer, - builder, - columnConfig - ); - ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); - capabilitiesBuilder.setDictionaryEncoded(true); - capabilitiesBuilder.setDictionaryValuesSorted(true); - capabilitiesBuilder.setDictionaryValuesUnique(true); - builder.setType(logicalType); - builder.setNestedCommonFormatColumnSupplier(supplier); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); - builder.setFilterable(true); - }); + return new VariantColumnDeserializer(); } if (logicalType.is(ValueType.STRING)) { - return ((buffer, builder, columnConfig) -> { - ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read( - byteOrder, - bitmapSerdeFactory, - buffer, - builder, - columnConfig - ); - ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); - capabilitiesBuilder.setDictionaryEncoded(true); - capabilitiesBuilder.setDictionaryValuesSorted(true); - capabilitiesBuilder.setDictionaryValuesUnique(true); - builder.setType(logicalType); - builder.setNestedCommonFormatColumnSupplier(supplier); - builder.setIndexSupplier(supplier, true, false); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); - builder.setFilterable(true); - }); + return new StringColumnDeserializer(); } if (logicalType.is(ValueType.LONG)) { - return ((buffer, builder, columnConfig) -> { - ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read( - byteOrder, - bitmapSerdeFactory, - buffer, - builder, - columnConfig - ); - ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); - capabilitiesBuilder.setDictionaryEncoded(true); - capabilitiesBuilder.setDictionaryValuesSorted(true); - capabilitiesBuilder.setDictionaryValuesUnique(true); - builder.setType(logicalType); - builder.setNestedCommonFormatColumnSupplier(supplier); - builder.setIndexSupplier(supplier, true, false); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); - builder.setFilterable(true); - }); + return new LongColumnDeserializer(); } if (logicalType.is(ValueType.DOUBLE)) { - return ((buffer, builder, columnConfig) -> { - ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read( - byteOrder, - bitmapSerdeFactory, - buffer, - builder, - columnConfig - ); - ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); - capabilitiesBuilder.setDictionaryEncoded(true); - capabilitiesBuilder.setDictionaryValuesSorted(true); - capabilitiesBuilder.setDictionaryValuesUnique(true); - builder.setType(logicalType); - builder.setNestedCommonFormatColumnSupplier(supplier); - builder.setIndexSupplier(supplier, true, false); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); - builder.setFilterable(true); - }); + return new DoubleColumnDeserializer(); } - return (buffer, builder, columnConfig) -> { - NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( - logicalType, - hasNulls, - buffer, - builder, - columnConfig, - bitmapSerdeFactory, - byteOrder - ); - ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); - capabilitiesBuilder.setDictionaryEncoded(true); - capabilitiesBuilder.setDictionaryValuesSorted(true); - capabilitiesBuilder.setDictionaryValuesUnique(true); - ColumnType simpleType = supplier.getLogicalType(); - ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType; - builder.setType(logicalType); - builder.setNestedCommonFormatColumnSupplier(supplier); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, hasNulls)); - builder.setFilterable(true); - }; + return new NestedColumnDeserializer(); } @JsonProperty @@ -260,6 +174,129 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde return bitmapSerdeFactory; } + private class StringColumnDeserializer implements Deserializer + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read( + byteOrder, + bitmapSerdeFactory, + buffer, + builder, + columnConfig + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + builder.setType(logicalType); + builder.setNestedCommonFormatColumnSupplier(supplier); + builder.setIndexSupplier(supplier, true, false); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); + builder.setFilterable(true); + } + } + + private class LongColumnDeserializer implements Deserializer + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read( + byteOrder, + bitmapSerdeFactory, + buffer, + builder, + columnConfig + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + builder.setType(logicalType); + builder.setNestedCommonFormatColumnSupplier(supplier); + builder.setIndexSupplier(supplier, true, false); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); + builder.setFilterable(true); + } + } + + private class DoubleColumnDeserializer implements Deserializer + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read( + byteOrder, + bitmapSerdeFactory, + buffer, + builder, + columnConfig + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + builder.setType(logicalType); + builder.setNestedCommonFormatColumnSupplier(supplier); + builder.setIndexSupplier(supplier, true, false); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); + builder.setFilterable(true); + } + } + + private class VariantColumnDeserializer implements Deserializer + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read( + logicalType, + byteOrder, + bitmapSerdeFactory, + buffer, + builder, + columnConfig + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + builder.setType(logicalType); + builder.setNestedCommonFormatColumnSupplier(supplier); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); + builder.setFilterable(true); + } + } + + private class NestedColumnDeserializer implements Deserializer + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( + logicalType, + hasNulls, + buffer, + builder, + columnConfig, + bitmapSerdeFactory, + byteOrder + ); + ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + ColumnType simpleType = supplier.getLogicalType(); + ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType; + builder.setType(logicalType); + builder.setNestedCommonFormatColumnSupplier(supplier); + builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, hasNulls)); + builder.setFilterable(true); + } + } + public static class SerializerBuilder { private ColumnType logicalType; diff --git a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java similarity index 80% rename from processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedColumnIndexSupplier.java rename to processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java index a617e3b409f..de9c791c7eb 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java @@ -25,6 +25,7 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.spatial.ImmutableRTree; import org.apache.druid.common.config.NullHandling; import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.column.DictionaryEncodedValueIndex; @@ -39,17 +40,16 @@ import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; import org.apache.druid.segment.column.SpatialIndex; import org.apache.druid.segment.column.StringEncodingStrategies; import org.apache.druid.segment.column.StringValueSetIndex; -import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; import javax.annotation.Nullable; import java.nio.ByteBuffer; -public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier +public class StringUtf8ColumnIndexSupplier> implements ColumnIndexSupplier { private final BitmapFactory bitmapFactory; - private final Supplier utf8Dictionary; + private final Supplier utf8Dictionary; @Nullable private final GenericIndexed bitmaps; @@ -57,17 +57,34 @@ public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier @Nullable private final ImmutableRTree indexedTree; - public StringFrontCodedColumnIndexSupplier( + private final ColumnConfig columnConfig; + private final int numRows; + + public StringUtf8ColumnIndexSupplier( BitmapFactory bitmapFactory, - Supplier utf8Dictionary, + Supplier utf8Dictionary, @Nullable GenericIndexed bitmaps, @Nullable ImmutableRTree indexedTree ) + { + this(bitmapFactory, utf8Dictionary, bitmaps, indexedTree, ColumnConfig.ALWAYS_USE_INDEXES, Integer.MAX_VALUE); + } + + public StringUtf8ColumnIndexSupplier( + BitmapFactory bitmapFactory, + Supplier utf8Dictionary, + @Nullable GenericIndexed bitmaps, + @Nullable ImmutableRTree indexedTree, + @Nullable ColumnConfig columnConfig, + int numRows + ) { this.bitmapFactory = bitmapFactory; this.bitmaps = bitmaps; this.utf8Dictionary = utf8Dictionary; this.indexedTree = indexedTree; + this.columnConfig = columnConfig; + this.numRows = numRows; } @Nullable @@ -90,7 +107,8 @@ public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier final BitmapColumnIndex nullIndex; final ByteBuffer firstValue = dict.get(0); if (NullHandling.isNullOrEquivalent(firstValue)) { - nullIndex = new SimpleImmutableBitmapIndex(singleThreadedBitmaps.get(0)); + ImmutableBitmap bitmap = singleThreadedBitmaps.get(0); + nullIndex = new SimpleImmutableBitmapIndex(bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap); } else { nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap()); } @@ -105,14 +123,18 @@ public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier return (T) new IndexedStringDruidPredicateIndex<>( bitmapFactory, new StringEncodingStrategies.Utf8ToStringIndexed(dict), - singleThreadedBitmaps + singleThreadedBitmaps, + columnConfig, + numRows ); } else if (clazz.equals(LexicographicalRangeIndex.class)) { return (T) new IndexedUtf8LexicographicalRangeIndex<>( bitmapFactory, dict, singleThreadedBitmaps, - dict.get(0) == null + dict.get(0) == null, + columnConfig, + numRows ); } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) || clazz.equals(DictionaryEncodedValueIndex.class)) { diff --git a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java similarity index 80% rename from processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java rename to processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java index 23bc28acff1..c0a8a0885e4 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/StringFrontCodedDictionaryEncodedColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java @@ -22,26 +22,25 @@ package org.apache.druid.segment.serde; import com.google.common.base.Supplier; import org.apache.druid.common.config.NullHandling; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.StringDictionaryEncodedColumn; import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.ColumnarMultiInts; -import org.apache.druid.segment.data.FrontCodedIndexed; +import org.apache.druid.segment.data.Indexed; import javax.annotation.Nullable; +import java.nio.ByteBuffer; /** - * {@link DictionaryEncodedColumnSupplier} but for columns using a {@link StringUtf8DictionaryEncodedColumn} - * instead of the traditional {@link StringDictionaryEncodedColumn} + * Supplier for {@link StringUtf8DictionaryEncodedColumn} */ -public class StringFrontCodedDictionaryEncodedColumnSupplier implements Supplier> +public class StringUtf8DictionaryEncodedColumnSupplier> implements Supplier> { - private final Supplier utf8Dictionary; + private final Supplier utf8Dictionary; private final @Nullable Supplier singleValuedColumn; private final @Nullable Supplier multiValuedColumn; - public StringFrontCodedDictionaryEncodedColumnSupplier( - Supplier utf8Dictionary, + public StringUtf8DictionaryEncodedColumnSupplier( + Supplier utf8Dictionary, @Nullable Supplier singleValuedColumn, @Nullable Supplier multiValuedColumn ) @@ -54,7 +53,7 @@ public class StringFrontCodedDictionaryEncodedColumnSupplier implements Supplier @Override public DictionaryEncodedColumn get() { - final FrontCodedIndexed suppliedUtf8Dictionary = utf8Dictionary.get(); + final TIndexed suppliedUtf8Dictionary = utf8Dictionary.get(); if (NullHandling.mustCombineNullAndEmptyInDictionary(suppliedUtf8Dictionary)) { return new StringUtf8DictionaryEncodedColumn( diff --git a/processing/src/test/java/org/apache/druid/query/DruidProcessingConfigTest.java b/processing/src/test/java/org/apache/druid/query/DruidProcessingConfigTest.java index e63d6e83d80..8d9f7137280 100644 --- a/processing/src/test/java/org/apache/druid/query/DruidProcessingConfigTest.java +++ b/processing/src/test/java/org/apache/druid/query/DruidProcessingConfigTest.java @@ -97,7 +97,6 @@ public class DruidProcessingConfigTest Assert.assertEquals(Integer.MAX_VALUE, config.poolCacheMaxCount()); Assert.assertEquals(NUM_PROCESSORS - 1, config.getNumThreads()); Assert.assertEquals(Math.max(2, config.getNumThreads() / 4), config.getNumMergeBuffers()); - Assert.assertEquals(0, config.columnCacheSizeBytes()); Assert.assertTrue(config.isFifo()); Assert.assertEquals(System.getProperty("java.io.tmpdir"), config.getTmpDir()); Assert.assertEquals(BUFFER_SIZE, config.intermediateComputeSizeBytes()); @@ -112,7 +111,6 @@ public class DruidProcessingConfigTest Assert.assertEquals(Integer.MAX_VALUE, config.poolCacheMaxCount()); Assert.assertTrue(config.getNumThreads() == 1); Assert.assertEquals(Math.max(2, config.getNumThreads() / 4), config.getNumMergeBuffers()); - Assert.assertEquals(0, config.columnCacheSizeBytes()); Assert.assertTrue(config.isFifo()); Assert.assertEquals(System.getProperty("java.io.tmpdir"), config.getTmpDir()); Assert.assertEquals(BUFFER_SIZE, config.intermediateComputeSizeBytes()); @@ -138,7 +136,6 @@ public class DruidProcessingConfigTest props.setProperty("druid.processing.buffer.sizeBytes", "1"); props.setProperty("druid.processing.buffer.poolCacheMaxCount", "1"); props.setProperty("druid.processing.numThreads", "256"); - props.setProperty("druid.processing.columnCache.sizeBytes", "1"); props.setProperty("druid.processing.fifo", "false"); props.setProperty("druid.processing.tmpDir", "/test/path"); @@ -156,7 +153,6 @@ public class DruidProcessingConfigTest Assert.assertEquals(1, config.poolCacheMaxCount()); Assert.assertEquals(256, config.getNumThreads()); Assert.assertEquals(64, config.getNumMergeBuffers()); - Assert.assertEquals(1, config.columnCacheSizeBytes()); Assert.assertFalse(config.isFifo()); Assert.assertEquals("/test/path", config.getTmpDir()); Assert.assertEquals(0, config.getNumInitalBuffersForIntermediatePool()); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index 564ff92198e..c60520fbfa4 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -168,11 +168,6 @@ public class AggregationTestHelper implements Closeable mapper, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); @@ -208,11 +203,6 @@ public class AggregationTestHelper implements Closeable mapper, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); @@ -260,11 +250,6 @@ public class AggregationTestHelper implements Closeable mapper, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); @@ -304,11 +289,6 @@ public class AggregationTestHelper implements Closeable mapper, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java index 3378a494b29..941505cf499 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -119,11 +119,6 @@ public class GroupByLimitPushDownInsufficientBufferTest extends InitializedNullH JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java index 831fe8e391e..2b50fcb4a5e 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -128,11 +128,6 @@ public class GroupByLimitPushDownMultiNodeMergeTest JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java index 2098a2fffe9..27c60c5d0fe 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java @@ -113,11 +113,6 @@ public class GroupByMultiSegmentTest JSON_MAPPER, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java index f04dacff2c8..def6630144d 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java @@ -74,6 +74,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; @@ -116,10 +117,7 @@ public class NestedQueryPushDownTest extends InitializedNullHandlingTest ExprMacroTable.nil() ) ); - INDEX_IO = new IndexIO( - JSON_MAPPER, - () -> 0 - ); + INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT); INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java index dc0dc72d255..4fdbe950c33 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java @@ -56,7 +56,7 @@ import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.StringDictionaryEncodedColumn; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.data.ObjectStrategy; @@ -434,7 +434,7 @@ public class SegmentAnalyzerTest extends InitializedNullHandlingTest ColumnHolder holder = EasyMock.createMock(ColumnHolder.class); EasyMock.expect(mockIndex.getColumnHolder("x")).andReturn(holder).atLeastOnce(); - StringDictionaryEncodedColumn dictionaryEncodedColumn = EasyMock.createMock(StringDictionaryEncodedColumn.class); + StringUtf8DictionaryEncodedColumn dictionaryEncodedColumn = EasyMock.createMock(StringUtf8DictionaryEncodedColumn.class); EasyMock.expect(holder.getColumn()).andReturn(dictionaryEncodedColumn).atLeastOnce(); dictionaryEncodedColumn.close(); diff --git a/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java index f875be0ce33..9d71c0c1e37 100644 --- a/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java @@ -28,7 +28,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.StringDictionaryEncodedColumn; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.serde.NoIndexesColumnIndexSupplier; import org.easymock.EasyMock; @@ -62,7 +62,7 @@ public class ColumnSelectorColumnIndexSelectorTest ColumnHolder holder = EasyMock.createMock(ColumnHolder.class); EasyMock.expect(index.getColumnHolder(STRING_DICTIONARY_COLUMN_NAME)).andReturn(holder).anyTimes(); - StringDictionaryEncodedColumn stringColumn = EasyMock.createMock(StringDictionaryEncodedColumn.class); + StringUtf8DictionaryEncodedColumn stringColumn = EasyMock.createMock(StringUtf8DictionaryEncodedColumn.class); EasyMock.expect(holder.getCapabilities()).andReturn( ColumnCapabilitiesImpl.createDefault() .setType(ColumnType.STRING) diff --git a/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java index 7715a7eeef6..227636b3eef 100644 --- a/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java @@ -27,6 +27,7 @@ import org.apache.druid.jackson.SegmentizerModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory; import org.apache.druid.segment.loading.SegmentLoadingException; @@ -58,7 +59,7 @@ public class CustomSegmentizerFactoryTest extends InitializedNullHandlingTest final ObjectMapper mapper = new DefaultObjectMapper(); mapper.registerModule(new SegmentizerModule()); mapper.registerSubtypes(new NamedType(CustomSegmentizerFactory.class, "customSegmentFactory")); - final IndexIO indexIO = new IndexIO(mapper, () -> 0); + final IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT); mapper.setInjectableValues( new InjectableValues.Std() diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index 8632a6ee01f..a04583a3b32 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -98,7 +98,7 @@ public class IndexBuilder public static IndexBuilder create() { - return new IndexBuilder(TestHelper.JSON_MAPPER, TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG); + return new IndexBuilder(TestHelper.JSON_MAPPER, ColumnConfig.ALWAYS_USE_INDEXES); } public static IndexBuilder create(ColumnConfig columnConfig) @@ -108,7 +108,7 @@ public class IndexBuilder public static IndexBuilder create(ObjectMapper jsonMapper) { - return new IndexBuilder(jsonMapper, TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG); + return new IndexBuilder(jsonMapper, ColumnConfig.ALWAYS_USE_INDEXES); } public static IndexBuilder create(ObjectMapper jsonMapper, ColumnConfig columnConfig) diff --git a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java index 90d235c4a03..a8d114af9d1 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java @@ -118,7 +118,7 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling @Test public void testV9LoaderThatIgnoresmptyColumns() throws IOException { - QueryableIndex queryableIndex = new V9IndexLoaderExceptEmptyColumns(TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG).load( + QueryableIndex queryableIndex = new V9IndexLoaderExceptEmptyColumns(ColumnConfig.ALWAYS_USE_INDEXES).load( segmentDir, TestHelper.makeJsonMapper(), false, diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index 5a80e18a69e..30c41bea70f 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -48,7 +48,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.StringDictionaryEncodedColumn; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.BitmapValues; @@ -548,12 +548,12 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest DictionaryEncodedColumn encodedColumn = (DictionaryEncodedColumn) index.getColumnHolder("dim2").getColumn(); Object obj; if (encodedColumn.hasMultipleValues()) { - Field field = StringDictionaryEncodedColumn.class.getDeclaredField("multiValueColumn"); + Field field = StringUtf8DictionaryEncodedColumn.class.getDeclaredField("multiValueColumn"); field.setAccessible(true); obj = field.get(encodedColumn); } else { - Field field = StringDictionaryEncodedColumn.class.getDeclaredField("column"); + Field field = StringUtf8DictionaryEncodedColumn.class.getDeclaredField("column"); field.setAccessible(true); obj = field.get(encodedColumn); diff --git a/processing/src/test/java/org/apache/druid/segment/TestHelper.java b/processing/src/test/java/org/apache/druid/segment/TestHelper.java index b68aafa7887..6aee1c433d9 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -62,26 +62,6 @@ import java.util.stream.IntStream; public class TestHelper { public static final ObjectMapper JSON_MAPPER = makeJsonMapper(); - public static final ColumnConfig NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 0; - } - - @Override - public double skipValueRangeIndexScale() - { - return 1.0; - } - - @Override - public double skipValuePredicateIndexScale() - { - return 1.0; - } - }; public static IndexMergerV9 getTestIndexMergerV9(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { @@ -90,7 +70,7 @@ public class TestHelper public static IndexIO getTestIndexIO() { - return getTestIndexIO(NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG); + return getTestIndexIO(ColumnConfig.ALWAYS_USE_INDEXES); } public static IndexIO getTestIndexIO(ColumnConfig columnConfig) diff --git a/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java b/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java index a9435f9a2f4..980b23e140c 100644 --- a/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java @@ -25,6 +25,7 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.segment.IndexIO.V9IndexLoader; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; @@ -119,7 +120,7 @@ public class V9IndexLoaderTest extends InitializedNullHandlingTest { private CannotDeserializeCountColumnV9IndexLoader() { - super(() -> 0); + super(ColumnConfig.DEFAULT); } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExtractionDimFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExtractionDimFilterTest.java index f9fd5171a41..692a2d499f4 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExtractionDimFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExtractionDimFilterTest.java @@ -42,7 +42,7 @@ import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; -import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier; +import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -116,13 +116,12 @@ public class ExtractionDimFilterTest extends InitializedNullHandlingTest public ColumnIndexSupplier getIndexSupplier(String column) { if ("foo".equals(column)) { - return new DictionaryEncodedStringIndexSupplier( + return new StringUtf8ColumnIndexSupplier<>( factory, - GenericIndexed.fromIterable(Collections.singletonList("foo1"), GenericIndexed.STRING_STRATEGY), GenericIndexed.fromIterable( Collections.singletonList(ByteBuffer.wrap(StringUtils.toUtf8("foo1"))), GenericIndexed.UTF8_STRATEGY - ), + )::singleThreaded, GenericIndexed.fromIterable(Collections.singletonList(foo1BitMap), serdeFactory.getObjectStrategy()), null ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java index f7525e3f9c4..3a7418421f5 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java @@ -32,7 +32,7 @@ import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.VSizeColumnarInts; import org.apache.druid.segment.data.VSizeColumnarMultiInts; import org.apache.druid.segment.selector.TestColumnValueSelector; -import org.apache.druid.segment.serde.DictionaryEncodedColumnSupplier; +import org.apache.druid.segment.serde.StringUtf8DictionaryEncodedColumnSupplier; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -67,8 +67,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes public void testDimensionProcessorMultiValuedDimensionMatchingValue() { // Emulate multi-valued dimension - final DictionaryEncodedColumnSupplier columnSupplier = new DictionaryEncodedColumnSupplier( - GenericIndexed.fromIterable(ImmutableList.of("v1", "v2", "v3"), GenericIndexed.STRING_STRATEGY), + final StringUtf8DictionaryEncodedColumnSupplier columnSupplier = new StringUtf8DictionaryEncodedColumnSupplier<>( GenericIndexed.fromIterable( ImmutableList.of( ByteBuffer.wrap(StringUtils.toUtf8("v1")), @@ -76,10 +75,9 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ByteBuffer.wrap(StringUtils.toUtf8("v3")) ), GenericIndexed.UTF8_STRATEGY - ), + )::singleThreaded, null, - () -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1}))), - 0 + () -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1}))) ); final ValueMatcher matcher = forSelector("v2") .makeDimensionProcessor(columnSupplier.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), true); @@ -90,8 +88,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes public void testDimensionProcessorMultiValuedDimensionNotMatchingValue() { // Emulate multi-valued dimension - final DictionaryEncodedColumnSupplier columnSupplier = new DictionaryEncodedColumnSupplier( - GenericIndexed.fromIterable(ImmutableList.of("v1", "v2", "v3"), GenericIndexed.STRING_STRATEGY), + final StringUtf8DictionaryEncodedColumnSupplier columnSupplier = new StringUtf8DictionaryEncodedColumnSupplier( GenericIndexed.fromIterable( ImmutableList.of( ByteBuffer.wrap(StringUtils.toUtf8("v1")), @@ -99,10 +96,9 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ByteBuffer.wrap(StringUtils.toUtf8("v3")) ), GenericIndexed.UTF8_STRATEGY - ), + )::singleThreaded, null, - () -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1}))), - 0 + () -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1}))) ); final ValueMatcher matcher = forSelector("v3") .makeDimensionProcessor(columnSupplier.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), true); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java index 98631d3dc9c..0f450a60117 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java @@ -26,7 +26,7 @@ import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.VSizeColumnarInts; import org.apache.druid.segment.data.VSizeColumnarMultiInts; -import org.apache.druid.segment.serde.DictionaryEncodedColumnSupplier; +import org.apache.druid.segment.serde.StringUtf8DictionaryEncodedColumnSupplier; import org.apache.druid.segment.vector.NilVectorSelector; import org.apache.druid.segment.vector.NoFilterVectorOffset; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -38,50 +38,44 @@ import java.nio.ByteBuffer; public class ValueMatchersTest extends InitializedNullHandlingTest { - private DictionaryEncodedColumnSupplier supplierSingleConstant; - private DictionaryEncodedColumnSupplier supplierSingle; - private DictionaryEncodedColumnSupplier supplierMulti; + private StringUtf8DictionaryEncodedColumnSupplier supplierSingleConstant; + private StringUtf8DictionaryEncodedColumnSupplier supplierSingle; + private StringUtf8DictionaryEncodedColumnSupplier supplierMulti; @Before public void setup() { - supplierSingleConstant = new DictionaryEncodedColumnSupplier( - GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY), + supplierSingleConstant = new StringUtf8DictionaryEncodedColumnSupplier<>( GenericIndexed.fromIterable( ImmutableList.of(ByteBuffer.wrap(StringUtils.toUtf8("value"))), GenericIndexed.UTF8_STRATEGY - ), + )::singleThreaded, () -> VSizeColumnarInts.fromArray(new int[]{0}), - null, - 0 + null ); - supplierSingle = new DictionaryEncodedColumnSupplier( - GenericIndexed.fromIterable(ImmutableList.of("value", "value2"), GenericIndexed.STRING_STRATEGY), + supplierSingle = new StringUtf8DictionaryEncodedColumnSupplier<>( GenericIndexed.fromIterable( ImmutableList.of( ByteBuffer.wrap(StringUtils.toUtf8("value")), ByteBuffer.wrap(StringUtils.toUtf8("value2")) ), GenericIndexed.UTF8_STRATEGY - ), + )::singleThreaded, () -> VSizeColumnarInts.fromArray(new int[]{0, 0, 1, 0, 1}), - null, - 0 + null ); - supplierMulti = new DictionaryEncodedColumnSupplier( - GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY), + supplierMulti = new StringUtf8DictionaryEncodedColumnSupplier<>( GenericIndexed.fromIterable( ImmutableList.of(ByteBuffer.wrap(StringUtils.toUtf8("value"))), GenericIndexed.UTF8_STRATEGY - ), + )::singleThreaded, null, () -> VSizeColumnarMultiInts.fromIterable( ImmutableList.of( VSizeColumnarInts.fromArray(new int[]{0, 0}), VSizeColumnarInts.fromArray(new int[]{0}) ) - ), - 0 + ) ); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index e579f0f0792..a5202260efe 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -188,7 +188,7 @@ public class JoinTestHelper public static IndexBuilder createFactIndexBuilder(final File tmpDir) throws IOException { - return createFactIndexBuilder(TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG, tmpDir, -1); + return createFactIndexBuilder(ColumnConfig.ALWAYS_USE_INDEXES, tmpDir, -1); } public static IndexBuilder createFactIndexBuilder( diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java index 8b11984354a..22ff1f3c5c2 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java @@ -47,6 +47,7 @@ import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.BaseColumn; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory; @@ -102,7 +103,7 @@ public class BroadcastSegmentIndexedTableTest extends InitializedNullHandlingTes { final ObjectMapper mapper = new DefaultObjectMapper(); mapper.registerModule(new SegmentizerModule()); - final IndexIO indexIO = new IndexIO(mapper, () -> 0); + final IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT); mapper.setInjectableValues( new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) diff --git a/processing/src/test/java/org/apache/druid/segment/loading/BroadcastJoinableMMappedQueryableSegmentizerFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/loading/BroadcastJoinableMMappedQueryableSegmentizerFactoryTest.java index 3f4f5ae9db7..9ce676f5ba6 100644 --- a/processing/src/test/java/org/apache/druid/segment/loading/BroadcastJoinableMMappedQueryableSegmentizerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/loading/BroadcastJoinableMMappedQueryableSegmentizerFactoryTest.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.TestIndex; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.join.table.BroadcastSegmentIndexedTable; import org.apache.druid.segment.join.table.IndexedTable; @@ -67,7 +68,7 @@ public class BroadcastJoinableMMappedQueryableSegmentizerFactoryTest extends Ini { final ObjectMapper mapper = new DefaultObjectMapper(); mapper.registerModule(new SegmentizerModule()); - final IndexIO indexIO = new IndexIO(mapper, () -> 0); + final IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT); mapper.setInjectableValues( new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) diff --git a/processing/src/test/java/org/apache/druid/segment/loading/SegmentizerFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/loading/SegmentizerFactoryTest.java index 9873ebb188d..584722bdcca 100644 --- a/processing/src/test/java/org/apache/druid/segment/loading/SegmentizerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/loading/SegmentizerFactoryTest.java @@ -46,11 +46,6 @@ public class SegmentizerFactoryTest mapper, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 777; - } } ); mapper.setInjectableValues( diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index 1182569c198..d6131b2d820 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -100,11 +100,6 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest private static final ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } @Override public double skipValueRangeIndexScale() diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java index fbe51d298dc..9503e55cc57 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java @@ -66,12 +66,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT private static final int ROW_COUNT = 10; static final ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } - @Override public double skipValueRangeIndexScale() { @@ -1470,12 +1464,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT { ColumnConfig twentyPercent = new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } - @Override public double skipValueRangeIndexScale() { diff --git a/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java index 87abedee1d7..7537dc53c92 100644 --- a/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java @@ -52,7 +52,7 @@ public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHan @Test public void testStringColumnWithNullValueSetIndex() throws IOException { - DictionaryEncodedStringIndexSupplier indexSupplier = makeStringWithNullsSupplier(); + StringUtf8ColumnIndexSupplier indexSupplier = makeStringWithNullsSupplier(); StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); Assert.assertNotNull(valueSetIndex); @@ -102,7 +102,7 @@ public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHan checkBitmap(bitmap); } - private DictionaryEncodedStringIndexSupplier makeStringWithNullsSupplier() throws IOException + private StringUtf8ColumnIndexSupplier makeStringWithNullsSupplier() throws IOException { ByteBuffer stringBuffer = ByteBuffer.allocate(1 << 12); ByteBuffer byteBuffer = ByteBuffer.allocate(1 << 12); @@ -164,10 +164,9 @@ public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHan writeToBuffer(bitmapsBuffer, bitmapWriter); GenericIndexed bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy()); - return new DictionaryEncodedStringIndexSupplier( + return new StringUtf8ColumnIndexSupplier<>( roaringFactory.getBitmapFactory(), - GenericIndexed.read(stringBuffer, GenericIndexed.STRING_STRATEGY), - GenericIndexed.read(byteBuffer, GenericIndexed.UTF8_STRATEGY), + GenericIndexed.read(byteBuffer, GenericIndexed.UTF8_STRATEGY)::singleThreaded, bitmaps, null ); diff --git a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java index df7a1029844..2c3455478d0 100644 --- a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java @@ -49,9 +49,7 @@ import java.nio.ByteBuffer; public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { - private static final String COLUMN_NAME = "missing"; private static final ByteBuffer EMPTY_BUFFER = ByteBuffer.allocate(0); - private static final ColumnConfig A_CONFIG = () -> 0; @Test public void testSerde() throws JsonProcessingException @@ -68,7 +66,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); final ColumnCapabilities columnCapabilities = builder.build().getCapabilities(); Assert.assertTrue(Types.is(columnCapabilities, ValueType.DOUBLE)); Assert.assertTrue(columnCapabilities.hasNulls().isTrue()); @@ -85,7 +83,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -107,7 +105,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -135,7 +133,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -154,7 +152,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -175,7 +173,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -199,7 +197,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); ColumnHolder holder = builder.build(); Assert.assertNull(holder.getIndexSupplier()); } diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java index 8253b27bb59..cacec12e666 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java @@ -38,7 +38,7 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.StringDictionaryEncodedColumn; +import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.ReadableOffset; @@ -95,7 +95,7 @@ public class DummyStringVirtualColumn implements VirtualColumn return DimensionSelector.constant(null); } - StringDictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn()); + StringUtf8DictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn()); DimensionSelector baseDimensionSelector = stringCol.makeDimensionSelector( offset, @@ -145,7 +145,7 @@ public class DummyStringVirtualColumn implements VirtualColumn return NilColumnValueSelector.instance(); } - StringDictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn()); + StringUtf8DictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn()); return stringCol.makeColumnValueSelector(offset); } else { return null; @@ -222,13 +222,13 @@ public class DummyStringVirtualColumn implements VirtualColumn return new byte[0]; } - private StringDictionaryEncodedColumn toStringDictionaryEncodedColumn(BaseColumn column) + private StringUtf8DictionaryEncodedColumn toStringDictionaryEncodedColumn(BaseColumn column) { - if (!(column instanceof StringDictionaryEncodedColumn)) { + if (!(column instanceof StringUtf8DictionaryEncodedColumn)) { throw new IAE("I can only work with StringDictionaryEncodedColumn"); } - return (StringDictionaryEncodedColumn) column; + return (StringUtf8DictionaryEncodedColumn) column; } private DimensionSelector disableValueMatchers(DimensionSelector base) diff --git a/server/src/test/java/org/apache/druid/guice/LocalDataStorageDruidModuleTest.java b/server/src/test/java/org/apache/druid/guice/LocalDataStorageDruidModuleTest.java index 2d1f6a477f5..8a7c1deb9ad 100644 --- a/server/src/test/java/org/apache/druid/guice/LocalDataStorageDruidModuleTest.java +++ b/server/src/test/java/org/apache/druid/guice/LocalDataStorageDruidModuleTest.java @@ -54,7 +54,7 @@ public class LocalDataStorageDruidModuleTest new LocalDataStorageDruidModule(), (Module) binder -> { binder.bind(new TypeLiteral>(){}).toInstance(ImmutableList.of()); - binder.bind(ColumnConfig.class).toInstance(() -> 0); + binder.bind(ColumnConfig.class).toInstance(ColumnConfig.DEFAULT); binder.bind(StorageLocationSelectorStrategy.class) .toInstance(new RandomStorageLocationSelectorStrategy(ImmutableList.of())); } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java index 858b989acf4..c1d2bf7a3a5 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java @@ -38,6 +38,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -183,10 +184,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable ); metrics = new FireDepartmentMetrics(); - IndexIO indexIO = new IndexIO( - objectMapper, - () -> 0 - ); + IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); IndexMergerV9 indexMerger = new IndexMergerV9( objectMapper, indexIO, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index cf8f2eea0d3..c6c2069b681 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -91,11 +91,6 @@ public class DefaultOfflineAppenderatorFactoryTest return 1; } - @Override - public int columnCacheSizeBytes() - { - return 25 * 1024 * 1024; - } } ); binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java index e2357d95c9b..552dc893baf 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java @@ -159,11 +159,6 @@ public class OpenAndClosedSegmentsAppenderatorTester implements AutoCloseable objectMapper, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index 413f315f50a..217c90116c3 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -157,11 +157,6 @@ public class StreamAppenderatorTester implements AutoCloseable objectMapper, new ColumnConfig() { - @Override - public int columnCacheSizeBytes() - { - return 0; - } } ); diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java index 3d54b279df1..97c000ea7b2 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerBroadcastJoinIndexedTableTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.TestIndex; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.join.BroadcastTableJoinableFactory; import org.apache.druid.segment.join.JoinConditionAnalysis; @@ -116,7 +117,7 @@ public class SegmentManagerBroadcastJoinIndexedTableTest extends InitializedNull new SimpleModule().registerSubtypes(new NamedType(LocalLoadSpec.class, "local")) ); - indexIO = new IndexIO(objectMapper, () -> 0); + indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); objectMapper.setInjectableValues( new InjectableValues.Std().addValue(LocalDataSegmentPuller.class, segmentPuller) .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java index 27abe091615..6eee7aef0f8 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; @@ -97,7 +98,7 @@ public class SegmentManagerThreadSafetyTest new SimpleModule().registerSubtypes(new NamedType(LocalLoadSpec.class, "local"), new NamedType(TestSegmentizerFactory.class, "test")) ) .setInjectableValues(new Std().addValue(LocalDataSegmentPuller.class, segmentPuller)); - indexIO = new IndexIO(objectMapper, () -> 0); + indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); segmentCacheDir = temporaryFolder.newFolder(); segmentDeepStorageDir = temporaryFolder.newFolder(); segmentCacheManager = new SegmentLocalCacheManager( diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index f78548adc4b..8c865af1664 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -761,11 +761,6 @@ public class DumpSegment extends GuiceRunnable return 1; } - @Override - public int columnCacheSizeBytes() - { - return 25 * 1024 * 1024; - } } ); binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); diff --git a/services/src/main/java/org/apache/druid/cli/ValidateSegments.java b/services/src/main/java/org/apache/druid/cli/ValidateSegments.java index de2b2bb6cb1..925d7c81a80 100644 --- a/services/src/main/java/org/apache/druid/cli/ValidateSegments.java +++ b/services/src/main/java/org/apache/druid/cli/ValidateSegments.java @@ -114,11 +114,6 @@ public class ValidateSegments extends GuiceRunnable return 1; } - @Override - public int columnCacheSizeBytes() - { - return 25 * 1024 * 1024; - } } ); binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); From 2d5b27358e0a1abefcc3c7ddeb3af613ddf88df5 Mon Sep 17 00:00:00 2001 From: Pranav Date: Mon, 3 Jul 2023 01:43:27 -0700 Subject: [PATCH 58/74] Logging the fieldName in the coerce exceptions (#14483) Logging the fieldName in the coerce exceptions --- .../apache/druid/msq/exec/ControllerImpl.java | 3 +- .../druid/msq/indexing/MSQControllerTask.java | 2 +- .../sql/calcite/run/NativeQueryMaker.java | 3 +- .../druid/sql/calcite/run/SqlResults.java | 80 ++++++++++++------- .../sql/calcite/CalciteScanSignatureTest.java | 29 +++++++ .../druid/sql/calcite/run/SqlResultsTest.java | 21 ++++- 6 files changed, 101 insertions(+), 37 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index eebc95077b3..1a8ef12cb2f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1471,7 +1471,8 @@ public class ControllerImpl implements Controller context.jsonMapper(), task.getSqlResultsContext(), value, - sqlTypeNames.get(i) + sqlTypeNames.get(i), + columnMappings.getOutputColumnName(i) ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index 02280525738..30047f227ff 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -82,7 +82,7 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery private final Map sqlQueryContext; /** - * Enables usage of {@link SqlResults#coerce(ObjectMapper, SqlResults.Context, Object, SqlTypeName)}. + * Enables usage of {@link SqlResults#coerce(ObjectMapper, SqlResults.Context, Object, SqlTypeName, String)}. */ @Nullable private final SqlResults.Context sqlResultsContext; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java index e2b30591901..e831960a390 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java @@ -241,7 +241,8 @@ public class NativeQueryMaker implements QueryMaker jsonMapper, sqlResultsContext, array[mapping[i]], - newTypes.get(i).getSqlTypeName() + newTypes.get(i).getSqlTypeName(), + originalFields.get(mapping[i]) ); } return newArray; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlResults.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlResults.java index ab400b97a3e..d314a3a4cc2 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlResults.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlResults.java @@ -28,8 +28,8 @@ import com.google.common.primitives.Ints; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.NlsString; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.math.expr.Evals; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.data.ComparableList; @@ -48,7 +48,7 @@ import java.util.Objects; import java.util.stream.Collectors; /** - * Holder for the utility method {@link #coerce(ObjectMapper, Context, Object, SqlTypeName)}. + * Holder for the utility method {@link #coerce(ObjectMapper, Context, Object, SqlTypeName, String)}. */ public class SqlResults { @@ -56,7 +56,8 @@ public class SqlResults final ObjectMapper jsonMapper, final Context context, final Object value, - final SqlTypeName sqlTypeName + final SqlTypeName sqlTypeName, + final String fieldName ) { final Object coercedValue; @@ -79,21 +80,21 @@ public class SqlResults final List valueStrings = ((Collection) maybeList) .stream() - .map(v -> (String) coerce(jsonMapper, context, v, sqlTypeName)) + .map(v -> (String) coerce(jsonMapper, context, v, sqlTypeName, fieldName)) .collect(Collectors.toList()); // Must stringify since the caller is expecting CHAR_TYPES. - coercedValue = coerceUsingObjectMapper(jsonMapper, valueStrings, sqlTypeName); + coercedValue = coerceUsingObjectMapper(jsonMapper, valueStrings, sqlTypeName, fieldName); } else { - throw cannotCoerce(value, sqlTypeName); + throw cannotCoerce(value, sqlTypeName, fieldName); } } } else if (value == null) { coercedValue = null; } else if (sqlTypeName == SqlTypeName.DATE) { - return Calcites.jodaToCalciteDate(coerceDateTime(value, sqlTypeName), context.getTimeZone()); + return Calcites.jodaToCalciteDate(coerceDateTime(value, sqlTypeName, fieldName), context.getTimeZone()); } else if (sqlTypeName == SqlTypeName.TIMESTAMP) { - return Calcites.jodaToCalciteTimestamp(coerceDateTime(value, sqlTypeName), context.getTimeZone()); + return Calcites.jodaToCalciteTimestamp(coerceDateTime(value, sqlTypeName, fieldName), context.getTimeZone()); } else if (sqlTypeName == SqlTypeName.BOOLEAN) { if (value instanceof Boolean) { coercedValue = value; @@ -102,7 +103,7 @@ public class SqlResults } else if (value instanceof Number) { coercedValue = Evals.asBoolean(((Number) value).longValue()); } else { - throw cannotCoerce(value, sqlTypeName); + throw cannotCoerce(value, sqlTypeName, fieldName); } } else if (sqlTypeName == SqlTypeName.INTEGER) { if (value instanceof String) { @@ -110,33 +111,33 @@ public class SqlResults } else if (value instanceof Number) { coercedValue = ((Number) value).intValue(); } else { - throw cannotCoerce(value, sqlTypeName); + throw cannotCoerce(value, sqlTypeName, fieldName); } } else if (sqlTypeName == SqlTypeName.BIGINT) { try { coercedValue = DimensionHandlerUtils.convertObjectToLong(value); } catch (Exception e) { - throw cannotCoerce(value, sqlTypeName); + throw cannotCoerce(value, sqlTypeName, fieldName); } } else if (sqlTypeName == SqlTypeName.FLOAT) { try { coercedValue = DimensionHandlerUtils.convertObjectToFloat(value); } catch (Exception e) { - throw cannotCoerce(value, sqlTypeName); + throw cannotCoerce(value, sqlTypeName, fieldName); } } else if (SqlTypeName.FRACTIONAL_TYPES.contains(sqlTypeName)) { try { coercedValue = DimensionHandlerUtils.convertObjectToDouble(value); } catch (Exception e) { - throw cannotCoerce(value, sqlTypeName); + throw cannotCoerce(value, sqlTypeName, fieldName); } } else if (sqlTypeName == SqlTypeName.OTHER) { // Complex type, try to serialize if we should, else print class name if (context.isSerializeComplexValues()) { - coercedValue = coerceUsingObjectMapper(jsonMapper, value, sqlTypeName); + coercedValue = coerceUsingObjectMapper(jsonMapper, value, sqlTypeName, fieldName); } else { coercedValue = value.getClass().getName(); } @@ -147,7 +148,7 @@ public class SqlResults } else if (value instanceof NlsString) { coercedValue = ((NlsString) value).getValue(); } else { - coercedValue = coerceUsingObjectMapper(jsonMapper, value, sqlTypeName); + coercedValue = coerceUsingObjectMapper(jsonMapper, value, sqlTypeName, fieldName); } } else { // the protobuf jdbc handler prefers lists (it actually can't handle java arrays as sql arrays, only java lists) @@ -155,11 +156,11 @@ public class SqlResults // here if needed coercedValue = maybeCoerceArrayToList(value, true); if (coercedValue == null) { - throw cannotCoerce(value, sqlTypeName); + throw cannotCoerce(value, sqlTypeName, fieldName); } } } else { - throw cannotCoerce(value, sqlTypeName); + throw cannotCoerce(value, sqlTypeName, fieldName); } return coercedValue; @@ -209,7 +210,7 @@ public class SqlResults return value; } - private static DateTime coerceDateTime(Object value, SqlTypeName sqlType) + private static DateTime coerceDateTime(final Object value, final SqlTypeName sqlType, final String fieldName) { final DateTime dateTime; @@ -220,7 +221,7 @@ public class SqlResults } else if (value instanceof DateTime) { dateTime = (DateTime) value; } else { - throw cannotCoerce(value, sqlType); + throw cannotCoerce(value, sqlType, fieldName); } return dateTime; } @@ -228,33 +229,52 @@ public class SqlResults private static String coerceUsingObjectMapper( final ObjectMapper jsonMapper, final Object value, - final SqlTypeName sqlTypeName + final SqlTypeName sqlTypeName, + final String fieldName ) { try { return jsonMapper.writeValueAsString(value); } catch (JsonProcessingException e) { - throw cannotCoerce(e, value, sqlTypeName); + throw cannotCoerce(e, value, sqlTypeName, fieldName); } } - private static IllegalStateException cannotCoerce( - final Throwable t, - final Object value, - final SqlTypeName sqlTypeName - ) + private static DruidException cannotCoerce(final Throwable t, final Object value, final SqlTypeName sqlTypeName, final String fieldName) { - return new ISE(t, "Cannot coerce [%s] to [%s]", value == null ? "null" : value.getClass().getName(), sqlTypeName); + return DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + t, + "Cannot coerce field [%s] from type [%s] to type [%s]", + fieldName, + value == null + ? "unknown" + : mapPriveArrayClassNameToReadableStrings(value.getClass().getName()), + sqlTypeName + ); } - private static IllegalStateException cannotCoerce(final Object value, final SqlTypeName sqlTypeName) + private static String mapPriveArrayClassNameToReadableStrings(String name) { - return cannotCoerce(null, value, sqlTypeName); + switch (name) { + case "[B": + return "Byte Array"; + case "[Z": + return "Boolean Array"; + default: + return name; + } + } + + private static DruidException cannotCoerce(final Object value, final SqlTypeName sqlTypeName, final String fieldName) + { + return cannotCoerce(null, value, sqlTypeName, fieldName); } /** - * Context for {@link #coerce(ObjectMapper, Context, Object, SqlTypeName)} + * Context for {@link #coerce(ObjectMapper, Context, Object, SqlTypeName, String)} */ public static class Context { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java index 13568f4e7a7..424a2e8895e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java @@ -81,6 +81,35 @@ public class CalciteScanSignatureTest extends BaseCalciteQueryTest ); } + @Test + public void testScanSignatureWithDimAsValuePrimitiveByteArr() + { + final Map context = new HashMap<>(QUERY_CONTEXT_DEFAULT); + testQuery( + "SELECT CAST(dim1 AS BIGINT) as dimX FROM foo2 limit 2", + ImmutableList.of( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE2) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("v0") + .virtualColumns(expressionVirtualColumn( + "v0", + "CAST(\"dim1\", 'LONG')", + ColumnType.LONG + )) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(context) + .limit(2) + .build() + ), + useDefault ? ImmutableList.of( + new Object[]{0L}, new Object[]{0L} + ) : ImmutableList.of( + new Object[]{null}, new Object[]{null} + ) + ); + } + @Override public SqlEngine createEngine( QueryLifecycleFactory qlf, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/run/SqlResultsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/run/SqlResultsTest.java index 954170fffeb..3c06e52238e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/run/SqlResultsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/run/SqlResultsTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSortedSet; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.data.ComparableList; @@ -171,6 +172,7 @@ public class SqlResultsTest extends InitializedNullHandlingTest assertCoerce(1L, true, SqlTypeName.BIGINT); assertCannotCoerce(Collections.emptyList(), SqlTypeName.BIGINT); + assertCannotCoerce(new byte[]{(byte) 0xe0, 0x4f}, SqlTypeName.BIGINT); } @Test @@ -222,6 +224,17 @@ public class SqlResultsTest extends InitializedNullHandlingTest assertCannotCoerce(new Object(), SqlTypeName.VARCHAR); } + @Test + public void testCoerceOfArrayOfPrimitives() + { + try { + assertCoerce("", new byte[1], SqlTypeName.BIGINT); + Assert.fail("Should throw an exception"); + } + catch (Exception e) { + Assert.assertEquals("Cannot coerce field [fieldName] from type [Byte Array] to type [BIGINT]", e.getMessage()); + } + } @Test public void testCoerceArrayFails() { @@ -251,16 +264,16 @@ public class SqlResultsTest extends InitializedNullHandlingTest Assert.assertEquals( StringUtils.format("Coerce [%s] to [%s]", toCoerce, typeName), expected, - SqlResults.coerce(jsonMapper, DEFAULT_CONTEXT, toCoerce, typeName) + SqlResults.coerce(jsonMapper, DEFAULT_CONTEXT, toCoerce, typeName, "fieldName") ); } private void assertCannotCoerce(Object toCoerce, SqlTypeName typeName) { - final IllegalStateException e = Assert.assertThrows( + final DruidException e = Assert.assertThrows( StringUtils.format("Coerce [%s] to [%s]", toCoerce, typeName), - IllegalStateException.class, - () -> SqlResults.coerce(jsonMapper, DEFAULT_CONTEXT, toCoerce, typeName) + DruidException.class, + () -> SqlResults.coerce(jsonMapper, DEFAULT_CONTEXT, toCoerce, typeName, "") ); MatcherAssert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Cannot coerce"))); From 27a70d569db7a0274e222d5ba1aa32cb587ce529 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 3 Jul 2023 15:20:14 +0530 Subject: [PATCH 59/74] Add page information to SqlStatementResource API (#14512) * Changes the get results API in SqlStatementResource to take a page number instead of row/offset. * Adds "pages" containing information on each page to the results status. * Update the "numRows" and "sizeInByes" to "numTotalRows" and "totalSizeInBytes" respectively, which are totalled across all pages. --- .../druid/msq/sql/entity/PageInformation.java | 103 ++++++++++++++++++ .../msq/sql/entity/ResultSetInformation.java | 64 ++++++----- .../sql/resources/SqlStatementResource.java | 52 +++++---- .../sql/SqlMsqStatementResourcePostTest.java | 6 +- .../msq/sql/SqlStatementResourceTest.java | 94 +++++----------- .../sql/entity/ResultSetInformationTest.java | 22 ++-- .../sql/entity/SqlStatementResultTest.java | 4 +- 7 files changed, 214 insertions(+), 131 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/PageInformation.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/PageInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/PageInformation.java new file mode 100644 index 00000000000..2754c52f1fe --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/PageInformation.java @@ -0,0 +1,103 @@ +/* + * 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.druid.msq.sql.entity; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Contains information about a single page in the results. + */ +public class PageInformation +{ + @Nullable + private final Long numRows; + @Nullable + private final Long sizeInBytes; + private final long id; + + @JsonCreator + public PageInformation( + @JsonProperty("numRows") @Nullable Long numRows, + @JsonProperty("sizeInBytes") @Nullable Long sizeInBytes, + @JsonProperty("id") long id + ) + { + this.numRows = numRows; + this.sizeInBytes = sizeInBytes; + this.id = id; + } + + @JsonProperty + @Nullable + public Long getNumRows() + { + return numRows; + } + + @JsonProperty + @Nullable + public Long getSizeInBytes() + { + return sizeInBytes; + } + + @JsonProperty + public long getId() + { + return id; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PageInformation that = (PageInformation) o; + return id == that.id && Objects.equals(numRows, that.numRows) && Objects.equals( + sizeInBytes, + that.sizeInBytes + ); + } + + @Override + public int hashCode() + { + return Objects.hash(numRows, sizeInBytes, id); + } + + @Override + public String toString() + { + return "PageInformation{" + + "numRows=" + numRows + + ", sizeInBytes=" + sizeInBytes + + ", id=" + id + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java index 43201fdac6e..e131fa85c73 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java @@ -32,50 +32,50 @@ public class ResultSetInformation { @Nullable - private final Long numRows; + private final Long numTotalRows; @Nullable - private final Long sizeInBytes; - + private final Long totalSizeInBytes; @Nullable private final ResultFormat resultFormat; - @Nullable private final List records; - @Nullable private final String dataSource; + @Nullable + private final List pages; @JsonCreator public ResultSetInformation( + @JsonProperty("numTotalRows") @Nullable Long numTotalRows, + @JsonProperty("totalSizeInBytes") @Nullable Long totalSizeInBytes, @JsonProperty("resultFormat") @Nullable ResultFormat resultFormat, - @JsonProperty("numRows") @Nullable Long numRows, - @JsonProperty("sizeInBytes") @Nullable Long sizeInBytes, @JsonProperty("dataSource") @Nullable String dataSource, - @JsonProperty("sampleRecords") @Nullable - List records + @JsonProperty("sampleRecords") @Nullable List records, + @JsonProperty("pages") @Nullable List pages ) { - this.numRows = numRows; - this.sizeInBytes = sizeInBytes; + this.numTotalRows = numTotalRows; + this.totalSizeInBytes = totalSizeInBytes; this.resultFormat = resultFormat; this.dataSource = dataSource; this.records = records; + this.pages = pages; } - @Nullable @JsonProperty + @Nullable @JsonInclude(JsonInclude.Include.NON_NULL) - public Long getNumRows() + public Long getNumTotalRows() { - return numRows; + return numTotalRows; } - @Nullable @JsonProperty + @Nullable @JsonInclude(JsonInclude.Include.NON_NULL) - public Long getSizeInBytes() + public Long getTotalSizeInBytes() { - return sizeInBytes; + return totalSizeInBytes; } @JsonProperty @@ -94,14 +94,21 @@ public class ResultSetInformation return dataSource; } - @Nullable @JsonProperty("sampleRecords") + @Nullable @JsonInclude(JsonInclude.Include.NON_NULL) public List getRecords() { return records; } + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getPages() + { + return pages; + } @Override public boolean equals(Object o) @@ -113,30 +120,31 @@ public class ResultSetInformation return false; } ResultSetInformation that = (ResultSetInformation) o; - return Objects.equals(numRows, that.numRows) - && Objects.equals(sizeInBytes, that.sizeInBytes) - && resultFormat == that.resultFormat - && Objects.equals(records, that.records) - && Objects.equals(dataSource, that.dataSource); + return Objects.equals(numTotalRows, that.numTotalRows) && Objects.equals( + totalSizeInBytes, + that.totalSizeInBytes + ) && resultFormat == that.resultFormat && Objects.equals(records, that.records) && Objects.equals( + dataSource, + that.dataSource + ) && Objects.equals(pages, that.pages); } @Override public int hashCode() { - return Objects.hash(numRows, sizeInBytes, resultFormat, records, dataSource); + return Objects.hash(numTotalRows, totalSizeInBytes, resultFormat, records, dataSource, pages); } @Override public String toString() { return "ResultSetInformation{" + - "totalRows=" + numRows + - ", totalSize=" + sizeInBytes + + "numTotalRows=" + numTotalRows + + ", totalSizeInBytes=" + totalSizeInBytes + ", resultFormat=" + resultFormat + ", records=" + records + ", dataSource='" + dataSource + '\'' + + ", pages=" + pages + '}'; } - } - diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 6ce5c780052..ce30284e593 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -21,6 +21,7 @@ package org.apache.druid.msq.sql.resources; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.io.CountingOutputStream; import com.google.common.util.concurrent.ListenableFuture; @@ -46,6 +47,7 @@ import org.apache.druid.msq.sql.MSQTaskQueryMaker; import org.apache.druid.msq.sql.MSQTaskSqlEngine; import org.apache.druid.msq.sql.SqlStatementState; import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; +import org.apache.druid.msq.sql.entity.PageInformation; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; import org.apache.druid.msq.util.SqlStatementResourceHelper; @@ -268,8 +270,7 @@ public class SqlStatementResource @Produces(MediaType.APPLICATION_JSON) public Response doGetResults( @PathParam("id") final String queryId, - @QueryParam("offset") Long offset, - @QueryParam("numRows") Long numberOfRows, + @QueryParam("page") Long page, @Context final HttpServletRequest req ) { @@ -284,27 +285,15 @@ public class SqlStatementResource } final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); - if (offset != null && offset < 0) { + if (page != null && page < 0) { return buildNonOkResponse( DruidException.forPersona(DruidException.Persona.USER) .ofCategory(DruidException.Category.INVALID_INPUT) .build( - "offset cannot be negative. Please pass a positive number." + "Page cannot be negative. Please pass a positive number." ) ); } - if (numberOfRows != null && numberOfRows < 0) { - return buildNonOkResponse( - DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - "numRows cannot be negative. Please pass a positive number." - ) - ); - } - - final long start = offset == null ? 0 : offset; - final long last = SqlStatementResourceHelper.getLastIndex(numberOfRows, start); TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId)); if (taskResponse == null) { @@ -343,8 +332,21 @@ public class SqlStatementResource if (!signature.isPresent()) { return Response.ok().build(); } - Optional> results = SqlStatementResourceHelper.getResults(SqlStatementResourceHelper.getPayload( - contactOverlord(overlordClient.taskReportAsMap(queryId)))); + + if (page != null && page > 0) { + // Results from task report are only present as one page. + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Page number is out of range of the results.") + ); + } + + Optional> results = SqlStatementResourceHelper.getResults( + SqlStatementResourceHelper.getPayload( + contactOverlord(overlordClient.taskReportAsMap(queryId)) + ) + ); return Response.ok((StreamingOutput) outputStream -> { CountingOutputStream os = new CountingOutputStream(outputStream); @@ -353,7 +355,7 @@ public class SqlStatementResource List rowSignature = signature.get(); writer.writeResponseStart(); - for (long k = start; k < Math.min(last, results.get().size()); k++) { + for (long k = 0; k < results.get().size(); k++) { writer.writeRowStart(); for (int i = 0; i < rowSignature.size(); i++) { writer.writeRowField( @@ -575,13 +577,19 @@ public class SqlStatementResource isSelectQuery ); return Optional.of(new ResultSetInformation( - null, - // since the rows can be sampled, get the number of rows from counters rowsAndSize.orElse(new Pair<>(null, null)).lhs, rowsAndSize.orElse(new Pair<>(null, null)).rhs, + null, dataSource, // only populate sample results in case a select query is successful - isSelectQuery ? SqlStatementResourceHelper.getResults(payload).orElse(null) : null + isSelectQuery ? SqlStatementResourceHelper.getResults(payload).orElse(null) : null, + ImmutableList.of( + new PageInformation( + rowsAndSize.orElse(new Pair<>(null, null)).lhs, + rowsAndSize.orElse(new Pair<>(null, null)).rhs, + 0 + ) + ) )); } else { return Optional.empty(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java index ceae64dcf70..51e10a93b2d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -30,6 +30,7 @@ import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; +import org.apache.druid.msq.sql.entity.PageInformation; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; import org.apache.druid.msq.sql.resources.SqlStatementResource; @@ -112,9 +113,9 @@ public class SqlMsqStatementResourcePostTest extends MSQTestBase ), MSQTestOverlordServiceClient.DURATION, new ResultSetInformation( - null, 6L, 316L, + null, MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, objectMapper.readValue( objectMapper.writeValueAsString( @@ -122,7 +123,8 @@ public class SqlMsqStatementResourcePostTest extends MSQTestBase new TypeReference>() { } - ) + ), + ImmutableList.of(new PageInformation(6L, 316L, 0)) ), null ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index b5d5addecd7..6a19f3f792e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -54,6 +54,7 @@ import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; import org.apache.druid.msq.indexing.report.MSQTaskReportTest; import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; +import org.apache.druid.msq.sql.entity.PageInformation; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; import org.apache.druid.msq.sql.resources.SqlStatementResource; @@ -650,7 +651,7 @@ public class SqlStatementResourceTest extends MSQTestBase ); assertExceptionMessage( - resource.doGetResults(ACCEPTED_SELECT_MSQ_QUERY, null, null, makeOkRequest()), + resource.doGetResults(ACCEPTED_SELECT_MSQ_QUERY, 0L, makeOkRequest()), StringUtils.format( "Query[%s] is currently in [%s] state. Please wait for it to complete.", ACCEPTED_SELECT_MSQ_QUERY, @@ -665,7 +666,6 @@ public class SqlStatementResourceTest extends MSQTestBase } @Test - public void testMSQSelectRunningQuery() { @@ -685,7 +685,7 @@ public class SqlStatementResourceTest extends MSQTestBase ); assertExceptionMessage( - resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, null, null, makeOkRequest()), + resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, 0L, makeOkRequest()), StringUtils.format( "Query[%s] is currently in [%s] state. Please wait for it to complete.", RUNNING_SELECT_MSQ_QUERY, @@ -717,12 +717,13 @@ public class SqlStatementResourceTest extends MSQTestBase MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, RESULT_ROWS.stream() .map(Arrays::asList) - .collect(Collectors.toList()) + .collect(Collectors.toList()), + ImmutableList.of(new PageInformation(null, null, 0L)) ), null ), response.getEntity()); - Response resultsResponse = resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, null, null, makeOkRequest()); + Response resultsResponse = resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, 0L, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); List> rows = new ArrayList<>(); @@ -736,43 +737,28 @@ public class SqlStatementResourceTest extends MSQTestBase resource.deleteQuery(FINISHED_SELECT_MSQ_QUERY, makeOkRequest()).getStatus() ); - Assert.assertEquals( - rows.subList(1, 2), - getResultRowsFromResponse(resource.doGetResults( - FINISHED_SELECT_MSQ_QUERY, - 1L, - null, - makeOkRequest() - )) - ); - Assert.assertEquals( - rows.subList(0, 1), - getResultRowsFromResponse(resource.doGetResults( - FINISHED_SELECT_MSQ_QUERY, - 0L, - 1L, - makeOkRequest() - )) - ); Assert.assertEquals( rows, getResultRowsFromResponse(resource.doGetResults( FINISHED_SELECT_MSQ_QUERY, 0L, - 3L, + makeOkRequest() + )) + ); + + Assert.assertEquals( + rows, + getResultRowsFromResponse(resource.doGetResults( + FINISHED_SELECT_MSQ_QUERY, + null, makeOkRequest() )) ); Assert.assertEquals( Response.Status.BAD_REQUEST.getStatusCode(), - resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, -1L, 3L, makeOkRequest()).getStatus() + resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, -1L, makeOkRequest()).getStatus() ); - Assert.assertEquals( - Response.Status.BAD_REQUEST.getStatusCode(), - resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, null, -1L, makeOkRequest()).getStatus() - ); - } @Test @@ -781,7 +767,7 @@ public class SqlStatementResourceTest extends MSQTestBase for (String queryID : ImmutableList.of(ERRORED_SELECT_MSQ_QUERY, ERRORED_INSERT_MSQ_QUERY)) { assertExceptionMessage(resource.doGetStatus(queryID, makeOkRequest()), FAILURE_MSG, Response.Status.OK); assertExceptionMessage( - resource.doGetResults(queryID, null, null, makeOkRequest()), + resource.doGetResults(queryID, 0L, makeOkRequest()), StringUtils.format( "Query[%s] failed. Hit status api for more details.", queryID @@ -797,7 +783,7 @@ public class SqlStatementResourceTest extends MSQTestBase } @Test - public void testFinishedInsertMSQQuery() throws Exception + public void testFinishedInsertMSQQuery() { Response response = resource.doGetStatus(FINISHED_INSERT_MSQ_QUERY, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); @@ -807,42 +793,17 @@ public class SqlStatementResourceTest extends MSQTestBase CREATED_TIME, null, 100L, - new ResultSetInformation(null, null, null, "test", null), + new ResultSetInformation(null, null, null, "test", null, ImmutableList.of(new PageInformation(null, null, 0))), null ), response.getEntity()); - Response resultsResponse = resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, null, makeOkRequest()); - Assert.assertEquals(Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); - - - Assert.assertNull(getResultRowsFromResponse(resource.doGetResults( - FINISHED_INSERT_MSQ_QUERY, - 1L, - null, - makeOkRequest() - ))); - Assert.assertNull(getResultRowsFromResponse(resource.doGetResults( - FINISHED_INSERT_MSQ_QUERY, - 0L, - 1L, - makeOkRequest() - ))); - Assert.assertNull(getResultRowsFromResponse(resource.doGetResults( - FINISHED_INSERT_MSQ_QUERY, - 0L, - 3L, - makeOkRequest() - ))); + Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, 0L, makeOkRequest()).getStatus()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, makeOkRequest()).getStatus()); Assert.assertEquals( Response.Status.BAD_REQUEST.getStatusCode(), - resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, -1L, 3L, makeOkRequest()).getStatus() + resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, -1L, makeOkRequest()).getStatus() ); - Assert.assertEquals( - Response.Status.BAD_REQUEST.getStatusCode(), - resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, -1L, makeOkRequest()).getStatus() - ); - } @Test @@ -850,7 +811,7 @@ public class SqlStatementResourceTest extends MSQTestBase { for (String queryID : ImmutableList.of(RUNNING_NON_MSQ_TASK, FAILED_NON_MSQ_TASK, FINISHED_NON_MSQ_TASK)) { assertNullResponse(resource.doGetStatus(queryID, makeOkRequest()), Response.Status.NOT_FOUND); - assertNullResponse(resource.doGetResults(queryID, null, null, makeOkRequest()), Response.Status.NOT_FOUND); + assertNullResponse(resource.doGetResults(queryID, 0L, makeOkRequest()), Response.Status.NOT_FOUND); assertNullResponse(resource.deleteQuery(queryID, makeOkRequest()), Response.Status.NOT_FOUND); } } @@ -874,7 +835,7 @@ public class SqlStatementResourceTest extends MSQTestBase ); assertExceptionMessage( - resource.doGetResults(ACCEPTED_INSERT_MSQ_TASK, null, null, makeOkRequest()), + resource.doGetResults(ACCEPTED_INSERT_MSQ_TASK, 0L, makeOkRequest()), StringUtils.format( "Query[%s] is currently in [%s] state. Please wait for it to complete.", ACCEPTED_INSERT_MSQ_TASK, @@ -907,7 +868,7 @@ public class SqlStatementResourceTest extends MSQTestBase ); assertExceptionMessage( - resource.doGetResults(RUNNING_INSERT_MSQ_QUERY, null, null, makeOkRequest()), + resource.doGetResults(RUNNING_INSERT_MSQ_QUERY, 0L, makeOkRequest()), StringUtils.format( "Query[%s] is currently in [%s] state. Please wait for it to complete.", RUNNING_INSERT_MSQ_QUERY, @@ -924,15 +885,12 @@ public class SqlStatementResourceTest extends MSQTestBase @Test public void forbiddenTests() { - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, - null, - null, + 1L, makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resource.deleteQuery(RUNNING_SELECT_MSQ_QUERY, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java index 14d04b1b76d..8e40d8daf1a 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java @@ -31,15 +31,19 @@ public class ResultSetInformationTest { public static final ObjectMapper MAPPER = new ObjectMapper(); - public static final ResultSetInformation RESULTS = new ResultSetInformation(ResultFormat.OBJECT, 1L, 1L, "ds", - ImmutableList.of( - ImmutableList.of("1"), - ImmutableList.of("2"), - ImmutableList.of("3") - ) + public static final ResultSetInformation RESULTS = new ResultSetInformation( + 1L, + 1L, + ResultFormat.OBJECT, + "ds", + ImmutableList.of( + ImmutableList.of("1"), + ImmutableList.of("2"), + ImmutableList.of("3") + ), + ImmutableList.of(new PageInformation(1L, 1L, 0)) ); - public static final String JSON_STRING = "{\"resultFormat\":\"object\",\"numRows\":1,\"sizeInBytes\":1,\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]]}"; - + public static final String JSON_STRING = "{\"numTotalRows\":1,\"totalSizeInBytes\":1,\"resultFormat\":\"object\",\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]],\"pages\":[{\"numRows\":1,\"sizeInBytes\":1,\"id\":0}]}"; @Test public void sanityTest() throws JsonProcessingException @@ -51,7 +55,7 @@ public class ResultSetInformationTest MAPPER.readValue(MAPPER.writeValueAsString(RESULTS), ResultSetInformation.class).hashCode() ); Assert.assertEquals( - "ResultSetInformation{totalRows=1, totalSize=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds'}", + "ResultSetInformation{numTotalRows=1, totalSizeInBytes=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds', pages=[PageInformation{numRows=1, sizeInBytes=1, id=0}]}", RESULTS.toString() ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java index a0be3afcf74..1409450c697 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java @@ -43,7 +43,7 @@ public class SqlStatementResultTest + "\"createdAt\":\"2023-05-31T12:00:00.000Z\"," + "\"schema\":[{\"name\":\"_time\",\"type\":\"TIMESTAMP\",\"nativeType\":\"LONG\"},{\"name\":\"alias\",\"type\":\"VARCHAR\",\"nativeType\":\"STRING\"},{\"name\":\"market\",\"type\":\"VARCHAR\",\"nativeType\":\"STRING\"}]," + "\"durationMs\":100," - + "\"result\":{\"resultFormat\":\"object\",\"numRows\":1,\"sizeInBytes\":1,\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]]}," + + "\"result\":{\"numTotalRows\":1,\"totalSizeInBytes\":1,\"resultFormat\":\"object\",\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]],\"pages\":[{\"numRows\":1,\"sizeInBytes\":1,\"id\":0}]}," + "\"errorDetails\":{\"error\":\"druidException\",\"errorCode\":\"QueryNotSupported\",\"persona\":\"USER\",\"category\":\"UNCATEGORIZED\",\"errorMessage\":\"QueryNotSupported\",\"context\":{}}}"; public static final SqlStatementResult SQL_STATEMENT_RESULT = new SqlStatementResult( @@ -87,7 +87,7 @@ public class SqlStatementResultTest + " createdAt=2023-05-31T12:00:00.000Z," + " sqlRowSignature=[ColumnNameAndTypes{colName='_time', sqlTypeName='TIMESTAMP', nativeTypeName='LONG'}, ColumnNameAndTypes{colName='alias', sqlTypeName='VARCHAR', nativeTypeName='STRING'}, ColumnNameAndTypes{colName='market', sqlTypeName='VARCHAR', nativeTypeName='STRING'}]," + " durationInMs=100," - + " resultSetInformation=ResultSetInformation{totalRows=1, totalSize=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds'}," + + " resultSetInformation=ResultSetInformation{numTotalRows=1, totalSizeInBytes=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds', pages=[PageInformation{numRows=1, sizeInBytes=1, id=0}]}," + " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}}}", SQL_STATEMENT_RESULT.toString() ); From c04a36d15b15ea0f2fea92d756a02335b5dbbb4b Mon Sep 17 00:00:00 2001 From: Tejaswini Bandlamudi <96047043+tejaswini-imply@users.noreply.github.com> Date: Mon, 3 Jul 2023 17:10:19 +0530 Subject: [PATCH 60/74] Run IntelliJ-inspections in parallel to static-checks & web-checks in GHA (#14515) Currently, IntelliJ-inspections are run sequentially w.r.t static-checks, thereby increasing build time. Moving IntelliJ-inspections to a separate job to improve builds time and get a quick insight into such issues early on. --- .github/workflows/static-checks.yml | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/.github/workflows/static-checks.yml b/.github/workflows/static-checks.yml index da4a7478402..b74e981add7 100644 --- a/.github/workflows/static-checks.yml +++ b/.github/workflows/static-checks.yml @@ -122,8 +122,25 @@ jobs: if: ${{ matrix.java == 'jdk8' }} run: ${MVN} spotbugs:check --fail-at-end -pl '!benchmarks' + intellij-inspections: + strategy: + fail-fast: false + runs-on: ubuntu-latest + steps: + - name: checkout branch + uses: actions/checkout@v3 + + - name: setup JDK8 + run: | + echo "JAVA_HOME=$JAVA_HOME_8_X64" >> $GITHUB_ENV + + - name: maven install + run: | + echo 'Running Maven install...' && + ${MVN} clean install -q -ff -pl '!distribution,!:druid-it-image,!:druid-it-cases' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} -T1C && + ${MVN} install -q -ff -pl 'distribution' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} + - name: intellij inspections - if: ${{ matrix.java == 'jdk8' }} run: | docker run --rm \ -v $(pwd):/project \ From 4ee7b14f5fe61839c68c121511c1307df1b367a5 Mon Sep 17 00:00:00 2001 From: Nhi Pham <56242907+demo-kratia@users.noreply.github.com> Date: Mon, 3 Jul 2023 13:50:25 -0700 Subject: [PATCH 61/74] update links in jupyter notebook (#14404) --- .../01-druidapi-package-intro.ipynb | 44 ++++++++++++++++++- .../03-query/00-using-sql-with-druidapi.ipynb | 23 +++++++++- .../notebooks/04-api/00-getting-started.ipynb | 35 ++++++++++++--- 3 files changed, 95 insertions(+), 7 deletions(-) diff --git a/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb index f3b104b14fd..6c943c4286b 100644 --- a/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb +++ b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb @@ -1,6 +1,7 @@ { "cells": [ { + "attachments": {}, "cell_type": "markdown", "id": "ce2efaaa", "metadata": {}, @@ -26,7 +27,7 @@ " ~ under the License.\n", " -->\n", "\n", - "This notebook provides a quick introduction to the Python wrapper around the [Druid REST API](api-tutorial.ipynb). This notebook assumes you are familiar with the basics of the REST API, and the [set of operations which Druid provides](https://druid.apache.org/docs/latest/operations/api-reference.html). This tutorial focuses on using Python to access those APIs rather than explaining the APIs themselves. The APIs themselves are covered in other notebooks that use the Python API.\n", + "This notebook provides a quick introduction to the Python wrapper around the [Druid REST API](api-tutorial.ipynb). This notebook assumes you are familiar with the basics of the REST API, and the [set of operations which Druid provides](https://druid.apache.org/docs/latest/api-reference/api-reference.html). This tutorial focuses on using Python to access those APIs rather than explaining the APIs themselves. The APIs themselves are covered in other notebooks that use the Python API.\n", "\n", "This tutorial works with Druid 25.0.0 or later.\n", "\n", @@ -52,6 +53,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "fb68a838", "metadata": {}, @@ -78,6 +80,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "8b4e774b", "metadata": {}, @@ -98,6 +101,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "be992774", "metadata": {}, @@ -116,6 +120,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "e803c9fe", "metadata": {}, @@ -134,6 +139,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "d78a6c35", "metadata": {}, @@ -152,6 +158,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "012b2e61", "metadata": {}, @@ -174,6 +181,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "d051bc5e", "metadata": {}, @@ -192,6 +200,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "b8261ab0", "metadata": {}, @@ -210,6 +219,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "ff311595", "metadata": {}, @@ -218,6 +228,17 @@ ] }, { + "cell_type": "code", + "execution_count": null, + "id": "616770ce", + "metadata": {}, + "outputs": [], + "source": [ + "display.tables()" + ] + }, + { + "attachments": {}, "cell_type": "markdown", "id": "7392e484", "metadata": {}, @@ -241,6 +262,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "c6c4e1d4", "metadata": {}, @@ -249,6 +271,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "f414d145", "metadata": {}, @@ -269,6 +292,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "7b944084", "metadata": {}, @@ -295,6 +319,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "937dc6b1", "metadata": {}, @@ -339,6 +364,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "481af1f2", "metadata": {}, @@ -357,6 +383,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "99f8db7b", "metadata": {}, @@ -365,6 +392,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "9e3be017", "metadata": {}, @@ -421,6 +449,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "ef4512f8", "metadata": {}, @@ -439,6 +468,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "11d9c95a", "metadata": {}, @@ -457,6 +487,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "936f57fb", "metadata": {}, @@ -475,6 +506,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "c1152f41", "metadata": {}, @@ -496,6 +528,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "c96fdcc6", "metadata": {}, @@ -517,6 +550,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "1deaf95f", "metadata": {}, @@ -538,6 +572,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "dcb8055f", "metadata": {}, @@ -556,6 +591,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "2654e72c", "metadata": {}, @@ -576,6 +612,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "837e08b0", "metadata": {}, @@ -617,6 +654,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "edc4ee39", "metadata": {}, @@ -647,6 +685,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "b661b29f", "metadata": {}, @@ -665,6 +704,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "5e789ca7", "metadata": {}, @@ -685,6 +725,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "7b9dc7e3", "metadata": {}, @@ -703,6 +744,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "ddaf0dc2", "metadata": {}, diff --git a/examples/quickstart/jupyter-notebooks/notebooks/03-query/00-using-sql-with-druidapi.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/03-query/00-using-sql-with-druidapi.ipynb index 4d9349d8f2b..47291d64dac 100644 --- a/examples/quickstart/jupyter-notebooks/notebooks/03-query/00-using-sql-with-druidapi.ipynb +++ b/examples/quickstart/jupyter-notebooks/notebooks/03-query/00-using-sql-with-druidapi.ipynb @@ -1,6 +1,7 @@ { "cells": [ { + "attachments": {}, "cell_type": "markdown", "id": "ad4e60b6", "metadata": { @@ -38,6 +39,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "8d6bbbcb", "metadata": { @@ -91,6 +93,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "e893ef7d-7136-442f-8bd9-31b5a5276518", "metadata": {}, @@ -142,6 +145,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "a141e962", "metadata": {}, @@ -160,6 +164,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "240b0ad5-48f2-4737-b12b-5fd5f98da300", "metadata": {}, @@ -188,6 +193,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "cbeb5a63", "metadata": { @@ -228,6 +234,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "59f41229", "metadata": {}, @@ -246,6 +253,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "c59ca797-dd91-442b-8d02-67b711b3fcc6", "metadata": {}, @@ -281,6 +289,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "f7cfdfae-ccba-49ba-a70f-63d0bd3527b2", "metadata": {}, @@ -293,6 +302,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "29c24856", "metadata": { @@ -321,6 +331,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "8cf212e5-fb3f-4206-acdd-46ef1da327ab", "metadata": { @@ -359,6 +370,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "5160db26-7e8d-40f7-8588-b7eabfc08355", "metadata": {}, @@ -396,6 +408,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "dd24470d-25c2-4031-a711-8477d69c9e94", "metadata": { @@ -408,6 +421,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "2baf21b9-74d1-4df6-862f-afbaeef1812b", "metadata": {}, @@ -437,6 +451,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "eab67db4-a0f3-4177-b5be-1fef355bf33f", "metadata": {}, @@ -467,6 +482,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "9c2e56af-3fdf-40f1-869b-822ac8aafbc8", "metadata": { @@ -489,6 +505,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "2de6e6ba-473c-4ef0-9739-a9472a4c7065", "metadata": {}, @@ -518,13 +535,14 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "407cf489-3947-4326-9d81-18b38abaee58", "metadata": {}, "source": [ "### TopN\n", "\n", - "The TopN native query type returns a sorted set of results for the values in a given dimension according to some criteria. TopN results are always computed in memory. In some cases, the TopN query type delivers approximate ranking and results. To prevent this, set the `useApproximateTopN` query context parameter to `false` when calling the [Druid SQL API](https://druid.apache.org/docs/latest/querying/sql-api.html). See [SQL query context](https://druid.apache.org/docs/latest/querying/sql-query-context.html) for more information.\n", + "The TopN native query type returns a sorted set of results for the values in a given dimension according to some criteria. TopN results are always computed in memory. In some cases, the TopN query type delivers approximate ranking and results. To prevent this, set the `useApproximateTopN` query context parameter to `false` when calling the [Druid SQL API](https://druid.apache.org/docs/latest/api-reference/sql-api.html). See [SQL query context](https://druid.apache.org/docs/latest/querying/sql-query-context.html) for more information.\n", "\n", "Druid SQL uses TopN for queries that meet the following criteria:\n", "- queries that GROUP BY a single expression\n", @@ -553,6 +571,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "38ee7d1d-2d47-4d36-b8c6-b0868c36c871", "metadata": {}, @@ -592,6 +611,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "eb5cfc5c-8f58-4e56-a218-7a55867e3e0c", "metadata": {}, @@ -625,6 +645,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "8fbfa1fa-2cde-46d5-8107-60bd436fb64e", "metadata": { diff --git a/examples/quickstart/jupyter-notebooks/notebooks/04-api/00-getting-started.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/04-api/00-getting-started.ipynb index f2e920a51fd..c703fc5fcfa 100644 --- a/examples/quickstart/jupyter-notebooks/notebooks/04-api/00-getting-started.ipynb +++ b/examples/quickstart/jupyter-notebooks/notebooks/04-api/00-getting-started.ipynb @@ -1,6 +1,7 @@ { "cells": [ { + "attachments": {}, "cell_type": "markdown", "id": "ad4e60b6", "metadata": { @@ -36,7 +37,7 @@ "\n", "Different [Druid server types](https://druid.apache.org/docs/latest/design/processes.html#server-types) are responsible for handling different APIs for the Druid services. For example, the Overlord service on the Master server provides the status of a task. You'll also interact the Broker service on the Query Server to see what datasources are available. And to run queries, you'll interact with the Broker. The Router service on the Query servers routes API calls.\n", "\n", - "For more information, see the [API reference](https://druid.apache.org/docs/latest/operations/api-reference.html), which is organized by server type.\n", + "For more information, see the [API reference](https://druid.apache.org/docs/latest/api-reference/api-reference.html), which is organized by server type.\n", "\n", "For work within other notebooks, prefer to use the [Python API](Python_API_Tutorial.ipynb) which is a notebook-friendly wrapper around the low-level API calls shown here.\n", "\n", @@ -52,6 +53,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "8d6bbbcb", "metadata": { @@ -76,6 +78,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "12c0e1c3", "metadata": {}, @@ -104,6 +107,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "a22c69c8", "metadata": {}, @@ -122,6 +126,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "2093ecf0-fb4b-405b-a216-094583580e0a", "metadata": {}, @@ -130,6 +135,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "29c24856", "metadata": { @@ -157,6 +163,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "1e853795", "metadata": {}, @@ -179,6 +186,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "de82029e", "metadata": {}, @@ -197,6 +205,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "cbeb5a63", "metadata": { @@ -231,6 +240,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "1917aace", "metadata": { @@ -243,7 +253,7 @@ "\n", "This tutorial uses the multi-stage query (MSQ) task engine and its `sql/task` endpoint to perform SQL-based ingestion.\n", "\n", - "To learn more about SQL-based ingestion, see [SQL-based ingestion](https://druid.apache.org/docs/latest/multi-stage-query/index.html). For information about the endpoint specifically, see [SQL-based ingestion and multi-stage query task API](https://druid.apache.org/docs/latest/multi-stage-query/api.html)." + "To learn more about SQL-based ingestion, see [SQL-based ingestion](https://druid.apache.org/docs/latest/multi-stage-query/index.html). For information about the endpoint specifically, see [SQL-based ingestion and multi-stage query task API](https://druid.apache.org/docs/latest/api-reference/sql-ingestion-api.html)." ] }, { @@ -258,6 +268,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "2168db3a", "metadata": { @@ -266,7 +277,7 @@ "source": [ "The example uses INSERT, but you could also use REPLACE INTO. In fact, if you have an existing datasource with the name `wikipedia_api`, you need to use REPLACE INTO instead. \n", "\n", - "The `/sql/task` endpoint accepts [SQL requests in the JSON-over-HTTP format](https://druid.apache.org/docs/latest/querying/sql-api.html#request-body) using the `query`, `context`, and `parameters` fields.\n", + "The `/sql/task` endpoint accepts [SQL requests in the JSON-over-HTTP format](https://druid.apache.org/docs/latest/api-reference/sql-api.html#request-body) using the `query`, `context`, and `parameters` fields.\n", "\n", "The query inserts data from an external source into a table named `wikipedia_api`. \n", "\n", @@ -307,6 +318,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "f7dcddd7", "metadata": {}, @@ -332,6 +344,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "6aa7f230", "metadata": {}, @@ -352,6 +365,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "9ba1821f", "metadata": { @@ -362,6 +376,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "f9cc2e45", "metadata": {}, @@ -380,6 +395,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "a983270d", "metadata": {}, @@ -401,6 +417,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "9f9c1b6b", "metadata": {}, @@ -420,6 +437,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "f17892d9-a8c1-43d6-890c-7d68cd792c72", "metadata": { @@ -458,6 +476,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "74a9b593", "metadata": {}, @@ -491,6 +510,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "3b55af57-9c79-4e45-a22c-438c1b94112e", "metadata": { @@ -528,6 +548,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "622f2158-75c9-4b12-bd8a-c92d30994c1f", "metadata": { @@ -551,6 +572,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "b2b366ad", "metadata": {}, @@ -574,6 +596,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "0df529d6", "metadata": {}, @@ -593,6 +616,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "950b2cc4-9935-497d-a3f5-e89afcc85965", "metadata": { @@ -648,6 +672,7 @@ ] }, { + "attachments": {}, "cell_type": "markdown", "id": "8fbfa1fa-2cde-46d5-8107-60bd436fb64e", "metadata": { @@ -658,8 +683,8 @@ "\n", "This tutorial covers the some of the basics related to the Druid API. To learn more about the kinds of things you can do, see the API documentation:\n", "\n", - "- [Druid SQL API](https://druid.apache.org/docs/latest/querying/sql-api.html)\n", - "- [API reference](https://druid.apache.org/docs/latest/operations/api-reference.html)\n", + "- [Druid SQL API](https://druid.apache.org/docs/latest/api-reference/api-reference.html)\n", + "- [API reference](https://druid.apache.org/docs/latest/api-reference/api-reference.html)\n", "\n", "You can also try out the [druid-client](https://github.com/paul-rogers/druid-client), a Python library for Druid created by Paul Rogers, a Druid contributor. A simplified version of that library is included with these tutorials. See [the Python API Tutorial](Python_API_Tutorial.ipynb) for an overview. That tutorial shows how to do the same tasks as this one, but in a simpler form: focusing on the Druid actions and not the mechanics of the REST API." ] From e2676c390ee663149ffd984db156eb8b0fb7bddc Mon Sep 17 00:00:00 2001 From: Rishabh Singh <6513075+findingrish@users.noreply.github.com> Date: Tue, 4 Jul 2023 12:21:35 +0530 Subject: [PATCH 62/74] Downgrade busybox version to fix k8s IT (#14518) --- distribution/docker/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/docker/Dockerfile b/distribution/docker/Dockerfile index 5f01c3dc09f..1c7933f09d3 100644 --- a/distribution/docker/Dockerfile +++ b/distribution/docker/Dockerfile @@ -49,7 +49,7 @@ RUN --mount=type=cache,target=/root/.m2 VERSION=$(mvn -B -q org.apache.maven.plu && tar -zxf ./distribution/target/apache-druid-${VERSION}-bin.tar.gz -C /opt \ && mv /opt/apache-druid-${VERSION} /opt/druid -FROM busybox:1.35.0-glibc as busybox +FROM busybox:1.34.1-glibc as busybox FROM gcr.io/distroless/java$JDK_VERSION-debian11 LABEL maintainer="Apache Druid Developers " From cc159f431722fc0a2c692f913fd4e016f862e40e Mon Sep 17 00:00:00 2001 From: Jakub Matyszewski Date: Tue, 4 Jul 2023 11:04:20 +0200 Subject: [PATCH 63/74] docs: k8s-jobs role needs batch apigroup (#14343) --- docs/development/extensions-contrib/k8s-jobs.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index f3e8d53bb35..7264694b6c1 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -125,6 +125,7 @@ metadata: rules: - apiGroups: - "" + - batch resources: - pods - configmaps From 609833c97bb44fee50f4fb84e44ad7c6cb053714 Mon Sep 17 00:00:00 2001 From: AmatyaAvadhanula Date: Wed, 5 Jul 2023 14:44:23 +0530 Subject: [PATCH 64/74] Do not emit negative lag because of stale offsets (#14292) The latest topic offsets are polled frequently and used to determine the lag based on the current offsets. However, when the offsets are stale (which can happen due to connection issues commonly), we may see a negative lag . This PR prevents emission of metrics when the offsets are stale and at least one of the partitions has a negative lag. --- .../supervisor/SeekableStreamSupervisor.java | 15 +++++++++++ .../SeekableStreamSupervisorStateTest.java | 25 +++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 0ae9aad9631..d99f84c5746 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -4220,6 +4220,21 @@ public abstract class SeekableStreamSupervisor 0 && partitionLags.values().stream().anyMatch(x -> x < 0)) { + // Log at most once every twenty supervisor runs to reduce noise in the logs + if ((staleMillis / getIoConfig().getPeriod().getMillis()) % 20 == 0) { + log.warn("Lag is negative and will not be emitted because topic offsets have become stale. " + + "This will not impact data processing. " + + "Offsets may become stale because of connectivity issues."); + } + return; + } + LagStats lagStats = computeLags(partitionLags); Map metricTags = spec.getContextValue(DruidMetrics.TAGS); for (Map.Entry entry : partitionLags.entrySet()) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index b8d5a556eef..a347541a4ef 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -62,6 +62,7 @@ import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.SeekableStreamExceptionEvent; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.SeekableStreamState; import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -1035,6 +1036,30 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport ); } + @Test + public void testStaleOffsetsNegativeLagNotEmitted() throws Exception + { + expectEmitterSupervisor(false); + + CountDownLatch latch = new CountDownLatch(1); + + final TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor( + latch, + TestEmittingTestSeekableStreamSupervisor.LAG, + // Record lag must not be emitted + ImmutableMap.of("0", 10L, "1", -100L), + null + ); + supervisor.start(); + // Forcibly set the offsets to be stale + supervisor.sequenceLastUpdated = DateTimes.nowUtc().minus(Integer.MAX_VALUE); + + latch.await(); + + supervisor.emitLag(); + Assert.assertEquals(0, emitter.getEvents().size()); + } + private List filterMetrics(List events, List whitelist) { List result = events.stream() From 50b7e5d20e551163c6ed45eb8c39617ff5634d7b Mon Sep 17 00:00:00 2001 From: Victoria Lim Date: Wed, 5 Jul 2023 12:29:47 -0700 Subject: [PATCH 65/74] docs: fix links (#14504) --- docs/api-reference/legacy-metadata-api.md | 4 ++-- docs/tutorials/tutorial-jupyter-docker.md | 2 +- docs/tutorials/tutorial-jupyter-index.md | 13 ++++++------- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/docs/api-reference/legacy-metadata-api.md b/docs/api-reference/legacy-metadata-api.md index bbb4ebb324d..609a7767f93 100644 --- a/docs/api-reference/legacy-metadata-api.md +++ b/docs/api-reference/legacy-metadata-api.md @@ -23,7 +23,7 @@ sidebar_label: Legacy metadata ~ under the License. --> -This document describes the legacy API endpoints to retrieve datasource metadata from Apache Druid. Use the [SQL metadata tables](/querying/sql-metadata-tables.md) to retrieve datasource metadata instead. +This document describes the legacy API endpoints to retrieve datasource metadata from Apache Druid. Use the [SQL metadata tables](../querying/sql-metadata-tables.md) to retrieve datasource metadata instead. ## Segment loading @@ -312,4 +312,4 @@ Returns the metrics of the datasource. `GET /druid/v2/datasources/{dataSourceName}/candidates?intervals={comma-separated-intervals}&numCandidates={numCandidates}` -Returns segment information lists including server locations for the given datasource and intervals. If "numCandidates" is not specified, it will return all servers for each interval. \ No newline at end of file +Returns segment information lists including server locations for the given datasource and intervals. If "numCandidates" is not specified, it will return all servers for each interval. diff --git a/docs/tutorials/tutorial-jupyter-docker.md b/docs/tutorials/tutorial-jupyter-docker.md index b5aa939db89..6886e3d6e7b 100644 --- a/docs/tutorials/tutorial-jupyter-docker.md +++ b/docs/tutorials/tutorial-jupyter-docker.md @@ -25,7 +25,7 @@ sidebar_label: "Docker for tutorials" Apache Druid provides a custom Jupyter container that contains the prerequisites -for all Jupyter-based Druid tutorials, as well as all of the tutorials themselves. +for all [Jupyter-based Druid tutorials](tutorial-jupyter-index.md), as well as all of the tutorials themselves. You can run the Jupyter container, as well as containers for Druid and Apache Kafka, using the Docker Compose file provided in the Druid GitHub repository. diff --git a/docs/tutorials/tutorial-jupyter-index.md b/docs/tutorials/tutorial-jupyter-index.md index c2e5eda6297..99279b64611 100644 --- a/docs/tutorials/tutorial-jupyter-index.md +++ b/docs/tutorials/tutorial-jupyter-index.md @@ -67,10 +67,9 @@ The notebooks are located in the [apache/druid repo](https://github.com/apache/d The links that follow are the raw GitHub URLs, so you can use them to download the notebook directly, such as with `wget`, or manually through your web browser. Note that if you save the file from your web browser, make sure to remove the `.txt` extension. -- [Introduction to the Druid REST API]( - https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/api-tutorial.ipynb) - walks you through some of the basics related to the Druid REST API and several endpoints. -- [Introduction to the Druid Python API]( - https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/Python_API_Tutorial.ipynb) - walks you through some of the basics related to the Druid API using the Python wrapper API. -- [Introduction to Druid SQL](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/sql-tutorial.ipynb) covers the basics of Druid SQL. +- [Introduction to the Druid REST API](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/notebooks/04-api/00-getting-started.ipynb) walks you through some of the + basics related to the Druid REST API and several endpoints. +- [Introduction to the Druid Python API](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb) walks you through some of the + basics related to the Druid API using the Python wrapper API. +- [Learn the basics of Druid SQL](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/notebooks/03-query/00-using-sql-with-druidapi.ipynb) introduces you to the unique aspects of Druid SQL with the primary focus on the SELECT statement. +- [Ingest and query data from Apache Kafka](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb) walks you through ingesting an event stream from Kafka. From f29a9faa94839bacfaad94bdeb4f439fe379dd18 Mon Sep 17 00:00:00 2001 From: Jonathan Wei Date: Wed, 5 Jul 2023 17:12:54 -0500 Subject: [PATCH 66/74] Better surfacing of invalid pattern errors for SQL REGEXP_EXTRACT function (#14505) --- .../RegexpExtractOperatorConversion.java | 36 +++++++++++++------ .../druid/sql/calcite/CalciteQueryTest.java | 21 +++++++++++ 2 files changed, 47 insertions(+), 10 deletions(-) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java index d131e5a66a0..0de60954130 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java @@ -24,6 +24,7 @@ import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.Expr; import org.apache.druid.query.extraction.RegexDimExtractionFn; @@ -33,6 +34,8 @@ import org.apache.druid.sql.calcite.expression.OperatorConversions; import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; import org.apache.druid.sql.calcite.planner.PlannerContext; +import java.util.regex.PatternSyntaxException; + public class RegexpExtractOperatorConversion implements SqlOperatorConversion { private static final SqlFunction SQL_FUNCTION = OperatorConversions @@ -74,16 +77,29 @@ public class RegexpExtractOperatorConversion implements SqlOperatorConversion if (arg.isSimpleExtraction() && patternExpr.isLiteral() && (indexExpr == null || indexExpr.isLiteral())) { final String pattern = (String) patternExpr.getLiteralValue(); - return arg.getSimpleExtraction().cascade( - new RegexDimExtractionFn( - // Undo the empty-to-null conversion from patternExpr parsing (patterns cannot be null, even in - // non-SQL-compliant null handling mode). - StringUtils.nullToEmptyNonDruidDataString(pattern), - indexExpr == null ? DEFAULT_INDEX : ((Number) indexExpr.getLiteralValue()).intValue(), - true, - null - ) - ); + + try { + return arg.getSimpleExtraction().cascade( + new RegexDimExtractionFn( + // Undo the empty-to-null conversion from patternExpr parsing (patterns cannot be null, even in + // non-SQL-compliant null handling mode). + StringUtils.nullToEmptyNonDruidDataString(pattern), + indexExpr == null ? DEFAULT_INDEX : ((Number) indexExpr.getLiteralValue()).intValue(), + true, + null + ) + ); + } + catch (PatternSyntaxException e) { + throw InvalidSqlInput.exception( + e, + StringUtils.format( + "An invalid pattern [%s] was provided for the REGEXP_EXTRACT function, error: [%s]", + e.getPattern(), + e.getMessage() + ) + ); + } } else { return null; } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index de6697388fc..54c8b360706 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -7511,6 +7511,27 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testRegexpExtractWithBadRegexPattern() + { + // Cannot vectorize due to extractionFn in dimension spec. + cannotVectorize(); + + expectedException.expect(DruidException.class); + expectedException.expectMessage( + "An invalid pattern [^(.))] was provided for the REGEXP_EXTRACT function, " + + "error: [Unmatched closing ')' near index 3\n^(.))\n ^]" + ); + + testQuery( + "SELECT DISTINCT\n" + + " REGEXP_EXTRACT(dim1, '^(.))', 1)\n" + + "FROM foo", + ImmutableList.of(), + ImmutableList.of() + ); + } + @Test public void testRegexpExtractFilterViaNotNullCheck() { From 78db7a44148bfba0acdfe6612daa3113914808b7 Mon Sep 17 00:00:00 2001 From: Soumyava <93540295+somu-imply@users.noreply.github.com> Date: Wed, 5 Jul 2023 20:29:35 -0700 Subject: [PATCH 67/74] A query in MSQ would issue wrong error code (#14531) with a RuntimeException. Now the RuntimeException is being replaced by an user facing DruidException of Invalid category which would allow calcite not to throw an uncategorized exception. --- .../external/ExternalOperatorConversion.java | 5 +++- .../sql/calcite/CalciteInsertDmlTest.java | 30 +++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalOperatorConversion.java index f2f667af4e3..26fdd514e8f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalOperatorConversion.java @@ -29,6 +29,7 @@ import org.apache.druid.catalog.model.table.BaseTableFunction; import org.apache.druid.catalog.model.table.ExternalTableSpec; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.column.RowSignature; @@ -120,7 +121,9 @@ public class ExternalOperatorConversion extends DruidExternTableMacroConversion ); } catch (JsonProcessingException e) { - throw new RuntimeException(e); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build(e, e.getMessage()); } } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index 1679b86fb06..71672ae8d51 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -1410,4 +1410,34 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest ) .verify(); } + + @Test + public void testErrorWithUnableToConstructColumnSignatureWithExtern() + { + final String sqlString = "insert into dst \n" + + "select time_parse(\"time\") as __time, * \n" + + "from table( \n" + + "extern(\n" + + "'{\"type\": \"s3\", \"uris\": [\\\"s3://imply-eng-datasets/qa/IngestionTest/wikipedia/files/wikiticker-2015-09-12-sampled.mini.json.gz\\\"]}',\n" + + "'{\"type\": \"json\"}',\n" + + "'[{\"name\": \"time\", \"type\": \"string\"}, {\"name\": \"channel\", \"type\": \"string\"}, {\"countryName\": \"string\"}]'\n" + + ")\n" + + ")\n" + + "partitioned by DAY\n" + + "clustered by channel"; + HashMap context = new HashMap<>(DEFAULT_CONTEXT); + context.put(PlannerContext.CTX_SQL_OUTER_LIMIT, 100); + testIngestionQuery().context(context).sql(sqlString) + .expectValidationError( + new DruidExceptionMatcher( + DruidException.Persona.USER, + DruidException.Category.INVALID_INPUT, + "general" + ) + .expectMessageContains( + "Cannot construct instance of `org.apache.druid.segment.column.ColumnSignature`, problem: `java.lang.NullPointerException`\n" + ) + ) + .verify(); + } } From a6547febaf8dcfb35f11073972317d7129035c98 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 6 Jul 2023 12:11:10 +0530 Subject: [PATCH 68/74] Remove unused coordinator dynamic configs (#14524) After #13197 , several coordinator configs are now redundant as they are not being used anymore, neither with `smartSegmentLoading` nor otherwise. Changes: - Remove dynamic configs `emitBalancingStats`: balancer error stats are always emitted, debug stats can be logged by using `debugDimensions` - `useBatchedSegmentSampler`, `percentOfSegmentsToConsiderPerMove`: batched segment sampling is always used - Add test to verify deserialization with unknown properties - Update `CoordinatorRunStats` to always track stats, this can be optimized later. --- .../BalancerStrategyBenchmark.java | 7 +- docs/configuration/index.md | 8 +- docs/operations/metrics.md | 2 - .../coordinator/CoordinatorDynamicConfig.java | 97 ---------- .../balancer/ReservoirSegmentSampler.java | 2 + .../loading/SegmentLoadingConfig.java | 18 +- .../stats/CoordinatorRunStats.java | 9 +- .../coordinator/stats/CoordinatorStat.java | 45 +++-- .../druid/server/coordinator/stats/Stats.java | 89 +++++----- .../coordinator/CoordinatorRunStatsTest.java | 166 ++++++++++-------- .../coordinator/duty/BalanceSegmentsTest.java | 5 +- .../CoordinatorSimulationBaseTest.java | 6 +- .../CoordinatorSimulationBuilder.java | 6 +- .../http/CoordinatorDynamicConfigTest.java | 129 +++----------- .../coordinator-dynamic-config.tsx | 47 ----- 15 files changed, 202 insertions(+), 434 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyBenchmark.java index 0bf5611998d..2afd99c8c38 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyBenchmark.java @@ -64,9 +64,6 @@ public class BalancerStrategyBenchmark private static final Interval TEST_SEGMENT_INTERVAL = Intervals.of("2012-03-15T00:00:00.000/2012-03-16T00:00:00.000"); private static final int NUMBER_OF_SERVERS = 20; - @Param({"default", "useBatchedSegmentSampler"}) - private String mode; - @Param({"10000", "100000", "1000000"}) private int numberOfSegments; @@ -79,9 +76,7 @@ public class BalancerStrategyBenchmark @Setup(Level.Trial) public void setup() { - if ("useBatchedSegmentSampler".equals(mode)) { - reservoirSize = maxSegmentsToMove; - } + reservoirSize = maxSegmentsToMove; List> segmentList = new ArrayList<>(NUMBER_OF_SERVERS); IntStream.range(0, NUMBER_OF_SERVERS).forEach(i -> segmentList.add(new ArrayList<>())); diff --git a/docs/configuration/index.md b/docs/configuration/index.md index b741728aad8..f0cf9c60112 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -931,11 +931,8 @@ A sample Coordinator dynamic config JSON object is shown below: "mergeBytesLimit": 100000000, "mergeSegmentsLimit" : 1000, "maxSegmentsToMove": 5, - "useBatchedSegmentSampler": false, - "percentOfSegmentsToConsiderPerMove": 100, "replicantLifetime": 15, "replicationThrottleLimit": 10, - "emitBalancingStats": false, "killDataSourceWhitelist": ["wikipedia", "testDatasource"], "decommissioningNodes": ["localhost:8182", "localhost:8282"], "decommissioningMaxPercentOfMaxSegmentsToMove": 70, @@ -953,19 +950,16 @@ Issuing a GET request at the same URL will return the spec that is currently in |`mergeBytesLimit`|The maximum total uncompressed size in bytes of segments to merge.|524288000L| |`mergeSegmentsLimit`|The maximum number of segments that can be in a single [append task](../ingestion/tasks.md).|100| |`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|100| -|`useBatchedSegmentSampler`|Deprecated. Boolean flag for whether or not we should use the Reservoir Sampling with a reservoir of size k instead of fixed size 1 to pick segments to move. This option can be enabled to speed up the sampling of segments to be balanced, especially if there is a large number of segments in the cluster or if there are too many segments to move.|true| -|`percentOfSegmentsToConsiderPerMove`|Deprecated. This will eventually be phased out by the batched segment sampler. You can enable the batched segment sampler now by setting the dynamic Coordinator config, `useBatchedSegmentSampler`, to `true`. Note that if you choose to enable the batched segment sampler, `percentOfSegmentsToConsiderPerMove` will no longer have any effect on balancing. If `useBatchedSegmentSampler == false`, this config defines the percentage of the total number of segments in the cluster that are considered every time a segment needs to be selected for a move. Druid orders servers by available capacity ascending (the least available capacity first) and then iterates over the servers. For each server, Druid iterates over the segments on the server, considering them for moving. The default config of 100% means that every segment on every server is a candidate to be moved. This should make sense for most small to medium-sized clusters. However, an admin may find it preferable to drop this value lower if they don't think that it is worthwhile to consider every single segment in the cluster each time it is looking for a segment to move.|100| |`replicantLifetime`|The maximum number of Coordinator runs for a segment to be replicated before we start alerting.|15| |`replicationThrottleLimit`|The maximum number of segments that can be in the replication queue of a historical tier at any given time.|500| |`balancerComputeThreads`|Thread pool size for computing moving cost of segments in segment balancing. Consider increasing this if you have a lot of segments and moving segments starts to get stuck.|1| -|`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false| |`killDataSourceWhitelist`|List of specific data sources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated data source names or a JSON array.|none| |`killPendingSegmentsSkipList`|List of data sources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated data sources or a JSON array.|none| |`maxSegmentsInNodeLoadingQueue`|The maximum number of segments allowed in the load queue of any given server. Use this parameter to load segments faster if, for example, the cluster contains slow-loading nodes or if there are too many segments to be replicated to a particular node (when faster loading is preferred to better segments distribution). The optimal value depends on the loading speed of segments, acceptable replication time and number of nodes. |500| |`useRoundRobinSegmentAssignment`|Boolean flag for whether segments should be assigned to historicals in a round robin fashion. When disabled, segment assignment is done using the chosen balancer strategy. When enabled, this can speed up segment assignments leaving balancing to move the segments to their optimal locations (based on the balancer strategy) lazily. |true| |`decommissioningNodes`| List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning' servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by `decommissioningMaxPercentOfMaxSegmentsToMove`.|none| |`decommissioningMaxPercentOfMaxSegmentsToMove`| Upper limit of segments the Coordinator can move from decommissioning servers to active non-decommissioning servers during a single run. This value is relative to the total maximum number of segments that can be moved at any given time based upon the value of `maxSegmentsToMove`.

    If `decommissioningMaxPercentOfMaxSegmentsToMove` is 0, the Coordinator does not move segments to decommissioning servers, effectively putting them in a type of "maintenance" mode. In this case, decommissioning servers do not participate in balancing or assignment by load rules. The Coordinator still considers segments on decommissioning servers as candidates to replicate on active servers.

    Decommissioning can stall if there are no available active servers to move the segments to. You can use the maximum percent of decommissioning segment movements to prioritize balancing or to decrease commissioning time to prevent active servers from being overloaded. The value must be between 0 and 100.|70| -|`pauseCoordination`| Boolean flag for whether or not the coordinator should execute its various duties of coordinating the cluster. Setting this to true essentially pauses all coordination work while allowing the API to remain up. Duties that are paused include all classes that implement the `CoordinatorDuty` Interface. Such duties include: Segment balancing, Segment compaction, Emission of metrics controlled by the dynamic coordinator config `emitBalancingStats`, Submitting kill tasks for unused segments (if enabled), Logging of used segments in the cluster, Marking of newly unused or overshadowed segments, Matching and execution of load/drop rules for used segments, Unloading segments that are no longer marked as used from Historical servers. An example of when an admin may want to pause coordination would be if they are doing deep storage maintenance on HDFS Name Nodes with downtime and don't want the coordinator to be directing Historical Nodes to hit the Name Node with API requests until maintenance is done and the deep store is declared healthy for use again. |false| +|`pauseCoordination`| Boolean flag for whether or not the coordinator should execute its various duties of coordinating the cluster. Setting this to true essentially pauses all coordination work while allowing the API to remain up. Duties that are paused include all classes that implement the `CoordinatorDuty` Interface. Such duties include: Segment balancing, Segment compaction, Submitting kill tasks for unused segments (if enabled), Logging of used segments in the cluster, Marking of newly unused or overshadowed segments, Matching and execution of load/drop rules for used segments, Unloading segments that are no longer marked as used from Historical servers. An example of when an admin may want to pause coordination would be if they are doing deep storage maintenance on HDFS Name Nodes with downtime and don't want the coordinator to be directing Historical Nodes to hit the Name Node with API requests until maintenance is done and the deep store is declared healthy for use again. |false| |`replicateAfterLoadTimeout`| Boolean flag for whether or not additional replication is needed for segments that have failed to load due to the expiry of `druid.coordinator.load.timeout`. If this is set to true, the coordinator will attempt to replicate the failed segment on a different historical server. This helps improve the segment availability if there are a few slow historicals in the cluster. However, the slow historical may still load the segment later and the coordinator may issue drop requests if the segment is over-replicated.|false| |`maxNonPrimaryReplicantsToLoad`|This is the maximum number of non-primary segment replicants to load per Coordination run. This number can be set to put a hard upper limit on the number of replicants loaded. It is a tool that can help prevent long delays in new data being available for query after events that require many non-primary replicants to be loaded by the cluster; such as a Historical node disconnecting from the cluster. The default value essentially means there is no limit on the number of replicants loaded per coordination cycle. If you want to use a non-default value for this config, you may want to start with it being `~20%` of the number of segments found on your Historical server with the most segments. You can use the Druid metric, `coordinator/time` with the filter `duty=org.apache.druid.server.coordinator.duty.RunRules` to see how different values of this config impact your Coordinator execution time.|`Integer.MAX_VALUE`| diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 5bc292dfd43..9003d0cb24f 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -325,8 +325,6 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina |`metadata/kill/datasource/count`|Total number of datasource metadata that were automatically deleted from metadata store per each Coordinator kill datasource duty run (Note: datasource metadata only exists for datasource created from supervisor). This metric can help adjust `druid.coordinator.kill.datasource.durationToRetain` configuration based on whether more or less datasource metadata need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.datasource.on` is set to true.| |Varies| |`init/serverview/time`|Time taken to initialize the coordinator server view.||Depends on the number of segments| -If `emitBalancingStats` is set to `true` in the Coordinator [dynamic configuration](../configuration/index.md#dynamic-configuration), then [log entries](../configuration/logging.md) for class `org.apache.druid.server.coordinator.duty.EmitClusterStatsAndMetrics` will have extra information on balancing decisions. - ## General Health ### Service Health diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index cd0e2b67e74..9a380294032 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -59,14 +59,9 @@ public class CoordinatorDynamicConfig private final long mergeBytesLimit; private final int mergeSegmentsLimit; private final int maxSegmentsToMove; - @Deprecated - private final double percentOfSegmentsToConsiderPerMove; - @Deprecated - private final boolean useBatchedSegmentSampler; private final int replicantLifetime; private final int replicationThrottleLimit; private final int balancerComputeThreads; - private final boolean emitBalancingStats; private final boolean useRoundRobinSegmentAssignment; private final boolean smartSegmentLoading; @@ -127,12 +122,9 @@ public class CoordinatorDynamicConfig @JsonProperty("mergeBytesLimit") long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") int mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") int maxSegmentsToMove, - @Deprecated @JsonProperty("percentOfSegmentsToConsiderPerMove") @Nullable Double percentOfSegmentsToConsiderPerMove, - @Deprecated @JsonProperty("useBatchedSegmentSampler") Boolean useBatchedSegmentSampler, @JsonProperty("replicantLifetime") int replicantLifetime, @JsonProperty("replicationThrottleLimit") int replicationThrottleLimit, @JsonProperty("balancerComputeThreads") int balancerComputeThreads, - @JsonProperty("emitBalancingStats") boolean emitBalancingStats, // Type is Object here so that we can support both string and list as Coordinator console can not send array of // strings in the update request. See https://github.com/apache/druid/issues/3055. // Keeping the legacy 'killDataSourceWhitelist' property name for backward compatibility. When the project is @@ -161,31 +153,9 @@ public class CoordinatorDynamicConfig this.maxSegmentsToMove = maxSegmentsToMove; this.smartSegmentLoading = Builder.valueOrDefault(smartSegmentLoading, Defaults.SMART_SEGMENT_LOADING); - if (percentOfSegmentsToConsiderPerMove == null) { - log.debug( - "percentOfSegmentsToConsiderPerMove was null! This is likely because your metastore does not " - + "reflect this configuration being added to Druid in a recent release. Druid is defaulting the value " - + "to the Druid default of %f. It is recommended that you re-submit your dynamic config with your " - + "desired value for percentOfSegmentsToConsideredPerMove", - Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE - ); - percentOfSegmentsToConsiderPerMove = Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE; - } - Preconditions.checkArgument( - percentOfSegmentsToConsiderPerMove > 0 && percentOfSegmentsToConsiderPerMove <= 100, - "'percentOfSegmentsToConsiderPerMove' should be between 1 and 100" - ); - this.percentOfSegmentsToConsiderPerMove = percentOfSegmentsToConsiderPerMove; - - this.useBatchedSegmentSampler = Builder.valueOrDefault( - useBatchedSegmentSampler, - Defaults.USE_BATCHED_SEGMENT_SAMPLER - ); - this.replicantLifetime = replicantLifetime; this.replicationThrottleLimit = replicationThrottleLimit; this.balancerComputeThreads = Math.max(balancerComputeThreads, 1); - this.emitBalancingStats = emitBalancingStats; this.specificDataSourcesToKillUnusedSegmentsIn = parseJsonStringOrArray(specificDataSourcesToKillUnusedSegmentsIn); this.dataSourcesToNotKillStalePendingSegmentsIn @@ -291,12 +261,6 @@ public class CoordinatorDynamicConfig return mergeBytesLimit; } - @JsonProperty - public boolean emitBalancingStats() - { - return emitBalancingStats; - } - @JsonProperty public int getMergeSegmentsLimit() { @@ -309,20 +273,6 @@ public class CoordinatorDynamicConfig return maxSegmentsToMove; } - @Deprecated - @JsonProperty - public double getPercentOfSegmentsToConsiderPerMove() - { - return percentOfSegmentsToConsiderPerMove; - } - - @Deprecated - @JsonProperty - public boolean useBatchedSegmentSampler() - { - return useBatchedSegmentSampler; - } - @JsonProperty public int getReplicantLifetime() { @@ -452,12 +402,9 @@ public class CoordinatorDynamicConfig ", mergeBytesLimit=" + mergeBytesLimit + ", mergeSegmentsLimit=" + mergeSegmentsLimit + ", maxSegmentsToMove=" + maxSegmentsToMove + - ", percentOfSegmentsToConsiderPerMove=" + percentOfSegmentsToConsiderPerMove + - ", useBatchedSegmentSampler=" + useBatchedSegmentSampler + ", replicantLifetime=" + replicantLifetime + ", replicationThrottleLimit=" + replicationThrottleLimit + ", balancerComputeThreads=" + balancerComputeThreads + - ", emitBalancingStats=" + emitBalancingStats + ", specificDataSourcesToKillUnusedSegmentsIn=" + specificDataSourcesToKillUnusedSegmentsIn + ", dataSourcesToNotKillStalePendingSegmentsIn=" + dataSourcesToNotKillStalePendingSegmentsIn + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + @@ -485,11 +432,8 @@ public class CoordinatorDynamicConfig && mergeBytesLimit == that.mergeBytesLimit && mergeSegmentsLimit == that.mergeSegmentsLimit && maxSegmentsToMove == that.maxSegmentsToMove - && percentOfSegmentsToConsiderPerMove == that.percentOfSegmentsToConsiderPerMove && decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove - && useBatchedSegmentSampler == that.useBatchedSegmentSampler && balancerComputeThreads == that.balancerComputeThreads - && emitBalancingStats == that.emitBalancingStats && replicantLifetime == that.replicantLifetime && replicationThrottleLimit == that.replicationThrottleLimit && replicateAfterLoadTimeout == that.replicateAfterLoadTimeout @@ -515,12 +459,9 @@ public class CoordinatorDynamicConfig mergeBytesLimit, mergeSegmentsLimit, maxSegmentsToMove, - percentOfSegmentsToConsiderPerMove, - useBatchedSegmentSampler, replicantLifetime, replicationThrottleLimit, balancerComputeThreads, - emitBalancingStats, maxSegmentsInNodeLoadingQueue, specificDataSourcesToKillUnusedSegmentsIn, dataSourcesToNotKillStalePendingSegmentsIn, @@ -543,12 +484,10 @@ public class CoordinatorDynamicConfig static final long MERGE_BYTES_LIMIT = 524_288_000L; static final int MERGE_SEGMENTS_LIMIT = 100; static final int MAX_SEGMENTS_TO_MOVE = 100; - static final double PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE = 100; static final int REPLICANT_LIFETIME = 15; static final int REPLICATION_THROTTLE_LIMIT = 500; static final int BALANCER_COMPUTE_THREADS = 1; static final boolean EMIT_BALANCING_STATS = false; - static final boolean USE_BATCHED_SEGMENT_SAMPLER = true; static final int MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 500; static final int DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT = 70; static final boolean PAUSE_COORDINATION = false; @@ -564,11 +503,8 @@ public class CoordinatorDynamicConfig private Long mergeBytesLimit; private Integer mergeSegmentsLimit; private Integer maxSegmentsToMove; - private Double percentOfSegmentsToConsiderPerMove; - private Boolean useBatchedSegmentSampler; private Integer replicantLifetime; private Integer replicationThrottleLimit; - private Boolean emitBalancingStats; private Integer balancerComputeThreads; private Object specificDataSourcesToKillUnusedSegmentsIn; private Object dataSourcesToNotKillStalePendingSegmentsIn; @@ -592,13 +528,9 @@ public class CoordinatorDynamicConfig @JsonProperty("mergeBytesLimit") @Nullable Long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") @Nullable Integer mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") @Nullable Integer maxSegmentsToMove, - @Deprecated @JsonProperty("percentOfSegmentsToConsiderPerMove") - @Nullable Double percentOfSegmentsToConsiderPerMove, - @Deprecated @JsonProperty("useBatchedSegmentSampler") Boolean useBatchedSegmentSampler, @JsonProperty("replicantLifetime") @Nullable Integer replicantLifetime, @JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit, @JsonProperty("balancerComputeThreads") @Nullable Integer balancerComputeThreads, - @JsonProperty("emitBalancingStats") @Nullable Boolean emitBalancingStats, @JsonProperty("killDataSourceWhitelist") @Nullable Object specificDataSourcesToKillUnusedSegmentsIn, @JsonProperty("killPendingSegmentsSkipList") @Nullable Object dataSourcesToNotKillStalePendingSegmentsIn, @JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue, @@ -617,12 +549,9 @@ public class CoordinatorDynamicConfig this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; - this.percentOfSegmentsToConsiderPerMove = percentOfSegmentsToConsiderPerMove; - this.useBatchedSegmentSampler = useBatchedSegmentSampler; this.replicantLifetime = replicantLifetime; this.replicationThrottleLimit = replicationThrottleLimit; this.balancerComputeThreads = balancerComputeThreads; - this.emitBalancingStats = emitBalancingStats; this.specificDataSourcesToKillUnusedSegmentsIn = specificDataSourcesToKillUnusedSegmentsIn; this.dataSourcesToNotKillStalePendingSegmentsIn = dataSourcesToNotKillStalePendingSegmentsIn; this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; @@ -660,20 +589,6 @@ public class CoordinatorDynamicConfig return this; } - @Deprecated - public Builder withPercentOfSegmentsToConsiderPerMove(double percentOfSegmentsToConsiderPerMove) - { - this.percentOfSegmentsToConsiderPerMove = percentOfSegmentsToConsiderPerMove; - return this; - } - - @Deprecated - public Builder withUseBatchedSegmentSampler(boolean useBatchedSegmentSampler) - { - this.useBatchedSegmentSampler = useBatchedSegmentSampler; - return this; - } - public Builder withSmartSegmentLoading(boolean smartSegmentLoading) { this.smartSegmentLoading = smartSegmentLoading; @@ -704,12 +619,6 @@ public class CoordinatorDynamicConfig return this; } - public Builder withEmitBalancingStats(boolean emitBalancingStats) - { - this.emitBalancingStats = emitBalancingStats; - return this; - } - public Builder withSpecificDataSourcesToKillUnusedSegmentsIn(Set dataSources) { this.specificDataSourcesToKillUnusedSegmentsIn = dataSources; @@ -772,12 +681,9 @@ public class CoordinatorDynamicConfig valueOrDefault(mergeBytesLimit, Defaults.MERGE_BYTES_LIMIT), valueOrDefault(mergeSegmentsLimit, Defaults.MERGE_SEGMENTS_LIMIT), valueOrDefault(maxSegmentsToMove, Defaults.MAX_SEGMENTS_TO_MOVE), - valueOrDefault(percentOfSegmentsToConsiderPerMove, Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE), - valueOrDefault(useBatchedSegmentSampler, Defaults.USE_BATCHED_SEGMENT_SAMPLER), valueOrDefault(replicantLifetime, Defaults.REPLICANT_LIFETIME), valueOrDefault(replicationThrottleLimit, Defaults.REPLICATION_THROTTLE_LIMIT), valueOrDefault(balancerComputeThreads, Defaults.BALANCER_COMPUTE_THREADS), - valueOrDefault(emitBalancingStats, Defaults.EMIT_BALANCING_STATS), specificDataSourcesToKillUnusedSegmentsIn, dataSourcesToNotKillStalePendingSegmentsIn, valueOrDefault(maxSegmentsInNodeLoadingQueue, Defaults.MAX_SEGMENTS_IN_NODE_LOADING_QUEUE), @@ -810,12 +716,9 @@ public class CoordinatorDynamicConfig valueOrDefault(mergeBytesLimit, defaults.getMergeBytesLimit()), valueOrDefault(mergeSegmentsLimit, defaults.getMergeSegmentsLimit()), valueOrDefault(maxSegmentsToMove, defaults.getMaxSegmentsToMove()), - valueOrDefault(percentOfSegmentsToConsiderPerMove, defaults.getPercentOfSegmentsToConsiderPerMove()), - valueOrDefault(useBatchedSegmentSampler, defaults.useBatchedSegmentSampler()), valueOrDefault(replicantLifetime, defaults.getReplicantLifetime()), valueOrDefault(replicationThrottleLimit, defaults.getReplicationThrottleLimit()), valueOrDefault(balancerComputeThreads, defaults.getBalancerComputeThreads()), - valueOrDefault(emitBalancingStats, defaults.emitBalancingStats()), valueOrDefault(specificDataSourcesToKillUnusedSegmentsIn, defaults.getSpecificDataSourcesToKillUnusedSegmentsIn()), valueOrDefault(dataSourcesToNotKillStalePendingSegmentsIn, defaults.getDataSourcesToNotKillStalePendingSegmentsIn()), valueOrDefault(maxSegmentsInNodeLoadingQueue, defaults.getMaxSegmentsInNodeLoadingQueue()), diff --git a/server/src/main/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSampler.java b/server/src/main/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSampler.java index 474347215aa..defa77a6b69 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSampler.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSampler.java @@ -43,6 +43,8 @@ public final class ReservoirSegmentSampler * * @param serverHolders Set of historicals to consider for picking segments * @param maxSegmentsToPick Maximum number of segments to pick + * @param segmentProvider Function to extract all movable segments from a + * {@link ServerHolder}. * @param broadcastDatasources Segments belonging to these datasources will not * be picked for balancing, since they should be * loaded on all servers anyway. diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java index fbf867fd91f..bb42e95eaae 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java @@ -39,7 +39,6 @@ public class SegmentLoadingConfig private final int percentDecommSegmentsToMove; private final boolean useRoundRobinSegmentAssignment; - private final boolean emitBalancingStats; /** * Creates a new SegmentLoadingConfig with recomputed coordinator config values from @@ -69,11 +68,10 @@ public class SegmentLoadingConfig 0, replicationThrottleLimit, Integer.MAX_VALUE, - dynamicConfig.getReplicantLifetime(), + 60, maxSegmentsToMove, 100, - true, - false + true ); } else { // Use the configured values @@ -84,8 +82,7 @@ public class SegmentLoadingConfig dynamicConfig.getReplicantLifetime(), dynamicConfig.getMaxSegmentsToMove(), dynamicConfig.getDecommissioningMaxPercentOfMaxSegmentsToMove(), - dynamicConfig.isUseRoundRobinSegmentAssignment(), - dynamicConfig.emitBalancingStats() + dynamicConfig.isUseRoundRobinSegmentAssignment() ); } } @@ -97,8 +94,7 @@ public class SegmentLoadingConfig int maxLifetimeInLoadQueue, int maxSegmentsToMove, int percentDecommSegmentsToMove, - boolean useRoundRobinSegmentAssignment, - boolean emitBalancingStats + boolean useRoundRobinSegmentAssignment ) { this.maxSegmentsInLoadQueue = maxSegmentsInLoadQueue; @@ -108,7 +104,6 @@ public class SegmentLoadingConfig this.maxSegmentsToMove = maxSegmentsToMove; this.percentDecommSegmentsToMove = percentDecommSegmentsToMove; this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment; - this.emitBalancingStats = emitBalancingStats; } public int getMaxSegmentsInLoadQueue() @@ -131,11 +126,6 @@ public class SegmentLoadingConfig return useRoundRobinSegmentAssignment; } - public boolean isEmitBalancingStats() - { - return emitBalancingStats; - } - public int getMaxLifetimeInLoadQueue() { return maxLifetimeInLoadQueue; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java index f8ea6cf98f8..c67d84c6818 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java @@ -89,7 +89,7 @@ public class CoordinatorRunStats /** * Builds a printable table of all the collected error, info and debug level - * stats (if applicable) with non-zero values. + * stats (if there are qualifying debugDimensions) with non-zero values. */ public String buildStatsTable() { @@ -183,13 +183,6 @@ public class CoordinatorRunStats public void add(CoordinatorStat stat, RowKey rowKey, long value) { - // Do not add a stat which will neither be emitted nor logged - if (!stat.shouldEmit() - && stat.getLevel() == CoordinatorStat.Level.DEBUG - && !hasDebugDimension(rowKey)) { - return; - } - allStats.computeIfAbsent(rowKey, d -> new Object2LongOpenHashMap<>()) .addTo(stat, value); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorStat.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorStat.java index a5d53f3735f..cad6652ca15 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorStat.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorStat.java @@ -29,27 +29,39 @@ public class CoordinatorStat private final Level level; /** - * Creates a new non-emitting, DEBUG level stat. + * Creates a new DEBUG level stat which is not emitted as a metric. + * + * @param shortName Unique name used while logging the stat */ - public CoordinatorStat(String shortStatName) + public static CoordinatorStat toDebugOnly(String shortName) { - this(shortStatName, null, Level.DEBUG); - } - - public CoordinatorStat(String shortName, Level level) - { - this(shortName, null, level); + return new CoordinatorStat(shortName, null, Level.DEBUG); } /** - * Creates a new emitting, DEBUG level stat. + * Creates a new DEBUG level stat which is also emitted as a metric. + * + * @param shortName Unique name used while logging the stat + * @param metricName Name to be used when emitting this stat as a metric */ - public CoordinatorStat(String shortStatName, String metricName) + public static CoordinatorStat toDebugAndEmit(String shortName, String metricName) { - this(shortStatName, metricName, Level.DEBUG); + return new CoordinatorStat(shortName, metricName, Level.DEBUG); } - public CoordinatorStat(String shortStatName, String metricName, Level level) + /** + * Creates a new stat of the specified level, which is also emitted as a metric. + * + * @param shortName Unique name used while logging the stat + * @param metricName Name to be used when emitting this stat as a metric + * @param level Logging level for this stat + */ + public static CoordinatorStat toLogAndEmit(String shortName, String metricName, Level level) + { + return new CoordinatorStat(shortName, metricName, level); + } + + private CoordinatorStat(String shortStatName, String metricName, Level level) { this.metricName = metricName; this.shortName = shortStatName; @@ -57,18 +69,25 @@ public class CoordinatorStat } /** - * Name of the metric emitted for this stat, if any. + * @return Metric name to be used when emitting this stat. {@code null} if + * this stat should not be emitted. */ public String getMetricName() { return metricName; } + /** + * Unique name used while logging this stat. + */ public String getShortName() { return shortName; } + /** + * Level of this stat, typically used for logging. + */ public Level getLevel() { return level; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java index 9864aa6b3ab..28f5c91049f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java @@ -28,114 +28,115 @@ public class Stats { // Decisions taken in a run public static final CoordinatorStat ASSIGNED - = new CoordinatorStat("assigned", "segment/assigned/count", CoordinatorStat.Level.INFO); + = CoordinatorStat.toLogAndEmit("assigned", "segment/assigned/count", CoordinatorStat.Level.INFO); public static final CoordinatorStat DROPPED - = new CoordinatorStat("dropped", "segment/dropped/count", CoordinatorStat.Level.INFO); + = CoordinatorStat.toLogAndEmit("dropped", "segment/dropped/count", CoordinatorStat.Level.INFO); public static final CoordinatorStat DELETED - = new CoordinatorStat("deleted", "segment/deleted/count", CoordinatorStat.Level.INFO); + = CoordinatorStat.toLogAndEmit("deleted", "segment/deleted/count", CoordinatorStat.Level.INFO); public static final CoordinatorStat MOVED - = new CoordinatorStat("moved", "segment/moved/count"); + = CoordinatorStat.toDebugAndEmit("moved", "segment/moved/count"); // Skipped decisions in a run public static final CoordinatorStat ASSIGN_SKIPPED - = new CoordinatorStat("assignSkip", "segment/assignSkipped/count"); + = CoordinatorStat.toDebugAndEmit("assignSkip", "segment/assignSkipped/count"); public static final CoordinatorStat DROP_SKIPPED - = new CoordinatorStat("dropSkip", "segment/dropSkipped/count"); + = CoordinatorStat.toDebugAndEmit("dropSkip", "segment/dropSkipped/count"); public static final CoordinatorStat MOVE_SKIPPED - = new CoordinatorStat("moveSkip", "segment/moveSkipped/count"); + = CoordinatorStat.toDebugAndEmit("moveSkip", "segment/moveSkipped/count"); // Current state of segments of a datasource public static final CoordinatorStat USED - = new CoordinatorStat("usedSegments", "segment/count"); + = CoordinatorStat.toDebugAndEmit("usedSegments", "segment/count"); public static final CoordinatorStat USED_BYTES - = new CoordinatorStat("usedSegmentBytes", "segment/size"); + = CoordinatorStat.toDebugAndEmit("usedSegmentBytes", "segment/size"); public static final CoordinatorStat UNDER_REPLICATED - = new CoordinatorStat("underreplicated", "segment/underReplicated/count"); + = CoordinatorStat.toDebugAndEmit("underreplicated", "segment/underReplicated/count"); public static final CoordinatorStat UNAVAILABLE - = new CoordinatorStat("unavailable", "segment/unavailable/count"); + = CoordinatorStat.toDebugAndEmit("unavailable", "segment/unavailable/count"); public static final CoordinatorStat UNNEEDED - = new CoordinatorStat("unneeded", "segment/unneeded/count"); + = CoordinatorStat.toDebugAndEmit("unneeded", "segment/unneeded/count"); public static final CoordinatorStat OVERSHADOWED - = new CoordinatorStat("overshadowed", "segment/overshadowed/count"); + = CoordinatorStat.toDebugAndEmit("overshadowed", "segment/overshadowed/count"); } public static class SegmentQueue { public static final CoordinatorStat NUM_TO_LOAD - = new CoordinatorStat("numToLoad", "segment/loadQueue/count"); + = CoordinatorStat.toDebugAndEmit("numToLoad", "segment/loadQueue/count"); public static final CoordinatorStat BYTES_TO_LOAD - = new CoordinatorStat("bytesToLoad", "segment/loadQueue/size"); + = CoordinatorStat.toDebugAndEmit("bytesToLoad", "segment/loadQueue/size"); public static final CoordinatorStat NUM_TO_DROP - = new CoordinatorStat("numToDrop", "segment/dropQueue/count"); + = CoordinatorStat.toDebugAndEmit("numToDrop", "segment/dropQueue/count"); public static final CoordinatorStat ASSIGNED_ACTIONS - = new CoordinatorStat("assignedActions", "segment/loadQueue/assigned"); + = CoordinatorStat.toDebugAndEmit("assignedActions", "segment/loadQueue/assigned"); public static final CoordinatorStat COMPLETED_ACTIONS - = new CoordinatorStat("successActions", "segment/loadQueue/success"); + = CoordinatorStat.toDebugAndEmit("successActions", "segment/loadQueue/success"); public static final CoordinatorStat FAILED_ACTIONS - = new CoordinatorStat("failedActions", "segment/loadQueue/failed", CoordinatorStat.Level.ERROR); + = CoordinatorStat.toLogAndEmit("failedActions", "segment/loadQueue/failed", CoordinatorStat.Level.ERROR); public static final CoordinatorStat CANCELLED_ACTIONS - = new CoordinatorStat("cancelledActions", "segment/loadQueue/cancelled"); + = CoordinatorStat.toDebugAndEmit("cancelledActions", "segment/loadQueue/cancelled"); } public static class Tier { public static final CoordinatorStat REQUIRED_CAPACITY - = new CoordinatorStat("reqdCap", "tier/required/capacity"); + = CoordinatorStat.toDebugAndEmit("reqdCap", "tier/required/capacity"); public static final CoordinatorStat TOTAL_CAPACITY - = new CoordinatorStat("totalCap", "tier/total/capacity"); + = CoordinatorStat.toDebugAndEmit("totalCap", "tier/total/capacity"); public static final CoordinatorStat REPLICATION_FACTOR - = new CoordinatorStat("maxRepFactor", "tier/replication/factor"); + = CoordinatorStat.toDebugAndEmit("maxRepFactor", "tier/replication/factor"); public static final CoordinatorStat HISTORICAL_COUNT - = new CoordinatorStat("numHistorical", "tier/historical/count"); + = CoordinatorStat.toDebugAndEmit("numHistorical", "tier/historical/count"); } public static class Compaction { public static final CoordinatorStat SUBMITTED_TASKS - = new CoordinatorStat("compactTasks", "compact/task/count"); + = CoordinatorStat.toDebugAndEmit("compactTasks", "compact/task/count"); public static final CoordinatorStat MAX_SLOTS - = new CoordinatorStat("compactMaxSlots", "compactTask/maxSlot/count"); + = CoordinatorStat.toDebugAndEmit("compactMaxSlots", "compactTask/maxSlot/count"); public static final CoordinatorStat AVAILABLE_SLOTS - = new CoordinatorStat("compactAvlSlots", "compactTask/availableSlot/count"); + = CoordinatorStat.toDebugAndEmit("compactAvlSlots", "compactTask/availableSlot/count"); public static final CoordinatorStat PENDING_BYTES - = new CoordinatorStat("compactPendingBytes", "segment/waitCompact/bytes"); + = CoordinatorStat.toDebugAndEmit("compactPendingBytes", "segment/waitCompact/bytes"); public static final CoordinatorStat COMPACTED_BYTES - = new CoordinatorStat("compactedBytes", "segment/compacted/bytes"); + = CoordinatorStat.toDebugAndEmit("compactedBytes", "segment/compacted/bytes"); public static final CoordinatorStat SKIPPED_BYTES - = new CoordinatorStat("compactSkipBytes", "segment/skipCompact/bytes"); + = CoordinatorStat.toDebugAndEmit("compactSkipBytes", "segment/skipCompact/bytes"); public static final CoordinatorStat PENDING_SEGMENTS - = new CoordinatorStat("compactPendingSeg", "segment/waitCompact/count"); + = CoordinatorStat.toDebugAndEmit("compactPendingSeg", "segment/waitCompact/count"); public static final CoordinatorStat COMPACTED_SEGMENTS - = new CoordinatorStat("compactedSeg", "segment/compacted/count"); + = CoordinatorStat.toDebugAndEmit("compactedSeg", "segment/compacted/count"); public static final CoordinatorStat SKIPPED_SEGMENTS - = new CoordinatorStat("compactSkipSeg", "segment/skipCompact/count"); + = CoordinatorStat.toDebugAndEmit("compactSkipSeg", "segment/skipCompact/count"); public static final CoordinatorStat PENDING_INTERVALS - = new CoordinatorStat("compactPendingIntv", "interval/waitCompact/count"); + = CoordinatorStat.toDebugAndEmit("compactPendingIntv", "interval/waitCompact/count"); public static final CoordinatorStat COMPACTED_INTERVALS - = new CoordinatorStat("compactedIntv", "interval/compacted/count"); + = CoordinatorStat.toDebugAndEmit("compactedIntv", "interval/compacted/count"); public static final CoordinatorStat SKIPPED_INTERVALS - = new CoordinatorStat("compactSkipIntv", "interval/skipCompact/count"); + = CoordinatorStat.toDebugAndEmit("compactSkipIntv", "interval/skipCompact/count"); } public static class CoordinatorRun { public static final CoordinatorStat DUTY_RUN_TIME - = new CoordinatorStat("dutyRunTime", "coordinator/time"); + = CoordinatorStat.toDebugAndEmit("dutyRunTime", "coordinator/time"); public static final CoordinatorStat GROUP_RUN_TIME - = new CoordinatorStat("groupRunTime", "coordinator/global/time"); + = CoordinatorStat.toDebugAndEmit("groupRunTime", "coordinator/global/time"); } public static class Balancer { - public static final CoordinatorStat COMPUTATION_ERRORS - = new CoordinatorStat("costComputeError", "segment/balancer/compute/error"); - public static final CoordinatorStat COMPUTATION_TIME - = new CoordinatorStat("costComputeTime", "segment/balancer/compute/time"); - public static final CoordinatorStat COMPUTATION_COUNT - = new CoordinatorStat("costComputeCount", "segment/balancer/compute/count"); + public static final CoordinatorStat COMPUTATION_ERRORS = CoordinatorStat.toLogAndEmit( + "costComputeError", + "segment/balancer/compute/error", + CoordinatorStat.Level.ERROR + ); + public static final CoordinatorStat COMPUTATION_TIME = CoordinatorStat.toDebugOnly("costComputeTime"); + public static final CoordinatorStat COMPUTATION_COUNT = CoordinatorStat.toDebugOnly("costComputeCount"); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java index c2c77f02cbf..1bf111a4f4c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRunStatsTest.java @@ -34,15 +34,6 @@ import java.util.Map; public class CoordinatorRunStatsTest { - private static final CoordinatorStat STAT_1 = new CoordinatorStat("stat1", "s1"); - private static final CoordinatorStat STAT_2 = new CoordinatorStat("stat2", "s2"); - private static final CoordinatorStat STAT_3 = new CoordinatorStat("stat3", "s3"); - - private static final CoordinatorStat DEBUG_STAT_1 - = new CoordinatorStat("debugStat1", CoordinatorStat.Level.DEBUG); - private static final CoordinatorStat DEBUG_STAT_2 - = new CoordinatorStat("debugStat2", CoordinatorStat.Level.DEBUG); - private CoordinatorRunStats stats; @Before @@ -60,56 +51,56 @@ public class CoordinatorRunStatsTest @Test public void testAdd() { - Assert.assertEquals(0, stats.get(STAT_1)); - stats.add(STAT_1, 1); - Assert.assertEquals(1, stats.get(STAT_1)); - stats.add(STAT_1, -11); - Assert.assertEquals(-10, stats.get(STAT_1)); + Assert.assertEquals(0, stats.get(Stat.ERROR_1)); + stats.add(Stat.ERROR_1, 1); + Assert.assertEquals(1, stats.get(Stat.ERROR_1)); + stats.add(Stat.ERROR_1, -11); + Assert.assertEquals(-10, stats.get(Stat.ERROR_1)); } @Test public void testAddForRowKey() { - stats.add(STAT_1, Key.TIER_1, 1); - stats.add(STAT_1, Key.TIER_2, 1); - stats.add(STAT_1, Key.TIER_1, -5); - stats.add(STAT_2, Key.TIER_1, 1); - stats.add(STAT_1, Key.TIER_2, 1); + stats.add(Stat.ERROR_1, Key.TIER_1, 1); + stats.add(Stat.ERROR_1, Key.TIER_2, 1); + stats.add(Stat.ERROR_1, Key.TIER_1, -5); + stats.add(Stat.INFO_1, Key.TIER_1, 1); + stats.add(Stat.ERROR_1, Key.TIER_2, 1); - Assert.assertFalse(stats.hasStat(STAT_3)); + Assert.assertFalse(stats.hasStat(Stat.INFO_2)); - Assert.assertEquals(-4, stats.get(STAT_1, Key.TIER_1)); - Assert.assertEquals(2, stats.get(STAT_1, Key.TIER_2)); - Assert.assertEquals(1, stats.get(STAT_2, Key.TIER_1)); + Assert.assertEquals(-4, stats.get(Stat.ERROR_1, Key.TIER_1)); + Assert.assertEquals(2, stats.get(Stat.ERROR_1, Key.TIER_2)); + Assert.assertEquals(1, stats.get(Stat.INFO_1, Key.TIER_1)); } @Test public void testGetSnapshotAndReset() { - stats.add(STAT_1, 1); - stats.add(STAT_2, 3); - stats.add(STAT_1, Key.TIER_1, 5); - stats.add(STAT_1, Key.DUTY_1, 7); + stats.add(Stat.ERROR_1, 1); + stats.add(Stat.INFO_1, 3); + stats.add(Stat.ERROR_1, Key.TIER_1, 5); + stats.add(Stat.ERROR_1, Key.DUTY_1, 7); final CoordinatorRunStats firstFlush = stats.getSnapshotAndReset(); - Assert.assertEquals(1, firstFlush.get(STAT_1)); - Assert.assertEquals(3, firstFlush.get(STAT_2)); - Assert.assertEquals(5, firstFlush.get(STAT_1, Key.TIER_1)); - Assert.assertEquals(7, firstFlush.get(STAT_1, Key.DUTY_1)); + Assert.assertEquals(1, firstFlush.get(Stat.ERROR_1)); + Assert.assertEquals(3, firstFlush.get(Stat.INFO_1)); + Assert.assertEquals(5, firstFlush.get(Stat.ERROR_1, Key.TIER_1)); + Assert.assertEquals(7, firstFlush.get(Stat.ERROR_1, Key.DUTY_1)); Assert.assertEquals(0, stats.rowCount()); - stats.add(STAT_1, 7); - stats.add(STAT_1, Key.TIER_1, 5); - stats.add(STAT_2, Key.DUTY_1, 3); - stats.add(STAT_3, Key.TIER_1, 1); + stats.add(Stat.ERROR_1, 7); + stats.add(Stat.ERROR_1, Key.TIER_1, 5); + stats.add(Stat.INFO_1, Key.DUTY_1, 3); + stats.add(Stat.INFO_2, Key.TIER_1, 1); final CoordinatorRunStats secondFlush = stats.getSnapshotAndReset(); - Assert.assertEquals(7, secondFlush.get(STAT_1)); - Assert.assertEquals(5, secondFlush.get(STAT_1, Key.TIER_1)); - Assert.assertEquals(3, secondFlush.get(STAT_2, Key.DUTY_1)); - Assert.assertEquals(1, secondFlush.get(STAT_3, Key.TIER_1)); + Assert.assertEquals(7, secondFlush.get(Stat.ERROR_1)); + Assert.assertEquals(5, secondFlush.get(Stat.ERROR_1, Key.TIER_1)); + Assert.assertEquals(3, secondFlush.get(Stat.INFO_1, Key.DUTY_1)); + Assert.assertEquals(1, secondFlush.get(Stat.INFO_2, Key.TIER_1)); Assert.assertEquals(0, stats.rowCount()); } @@ -117,38 +108,38 @@ public class CoordinatorRunStatsTest @Test public void testUpdateMax() { - stats.updateMax(STAT_1, Key.TIER_1, 2); - stats.updateMax(STAT_1, Key.TIER_1, 6); - stats.updateMax(STAT_1, Key.TIER_1, 5); + stats.updateMax(Stat.ERROR_1, Key.TIER_1, 2); + stats.updateMax(Stat.ERROR_1, Key.TIER_1, 6); + stats.updateMax(Stat.ERROR_1, Key.TIER_1, 5); - stats.updateMax(STAT_2, Key.TIER_1, 5); - stats.updateMax(STAT_2, Key.TIER_1, 4); - stats.updateMax(STAT_2, Key.TIER_1, 5); + stats.updateMax(Stat.INFO_1, Key.TIER_1, 5); + stats.updateMax(Stat.INFO_1, Key.TIER_1, 4); + stats.updateMax(Stat.INFO_1, Key.TIER_1, 5); - stats.updateMax(STAT_1, Key.TIER_2, 7); - stats.updateMax(STAT_1, Key.TIER_2, 9); - stats.updateMax(STAT_1, Key.TIER_2, 10); + stats.updateMax(Stat.ERROR_1, Key.TIER_2, 7); + stats.updateMax(Stat.ERROR_1, Key.TIER_2, 9); + stats.updateMax(Stat.ERROR_1, Key.TIER_2, 10); - Assert.assertFalse(stats.hasStat(STAT_3)); + Assert.assertFalse(stats.hasStat(Stat.INFO_2)); - Assert.assertEquals(6, stats.get(STAT_1, Key.TIER_1)); - Assert.assertEquals(5, stats.get(STAT_2, Key.TIER_1)); - Assert.assertEquals(10, stats.get(STAT_1, Key.TIER_2)); + Assert.assertEquals(6, stats.get(Stat.ERROR_1, Key.TIER_1)); + Assert.assertEquals(5, stats.get(Stat.INFO_1, Key.TIER_1)); + Assert.assertEquals(10, stats.get(Stat.ERROR_1, Key.TIER_2)); } @Test public void testAddToDutyStat() { - stats.add(STAT_1, Key.DUTY_1, 1); - stats.add(STAT_1, Key.DUTY_2, 1); - stats.add(STAT_1, Key.DUTY_1, -5); - stats.add(STAT_2, Key.DUTY_1, 1); - stats.add(STAT_1, Key.DUTY_2, 1); + stats.add(Stat.ERROR_1, Key.DUTY_1, 1); + stats.add(Stat.ERROR_1, Key.DUTY_2, 1); + stats.add(Stat.ERROR_1, Key.DUTY_1, -5); + stats.add(Stat.INFO_1, Key.DUTY_1, 1); + stats.add(Stat.ERROR_1, Key.DUTY_2, 1); - Assert.assertFalse(stats.hasStat(STAT_3)); - Assert.assertEquals(-4, stats.get(STAT_1, Key.DUTY_1)); - Assert.assertEquals(2, stats.get(STAT_1, Key.DUTY_2)); - Assert.assertEquals(1, stats.get(STAT_2, Key.DUTY_1)); + Assert.assertFalse(stats.hasStat(Stat.INFO_2)); + Assert.assertEquals(-4, stats.get(Stat.ERROR_1, Key.DUTY_1)); + Assert.assertEquals(2, stats.get(Stat.ERROR_1, Key.DUTY_2)); + Assert.assertEquals(1, stats.get(Stat.INFO_1, Key.DUTY_1)); } @Test @@ -161,13 +152,13 @@ public class CoordinatorRunStatsTest ); expected.forEach( (duty, count) -> - stats.add(STAT_1, RowKey.of(Dimension.DUTY, duty), count) + stats.add(Stat.ERROR_1, RowKey.of(Dimension.DUTY, duty), count) ); final Map actual = new HashMap<>(); stats.forEachStat( (stat, rowKey, value) -> { - if (stat.equals(STAT_1)) { + if (stat.equals(Stat.ERROR_1)) { actual.put(rowKey.getValues().get(Dimension.DUTY), value); } } @@ -176,24 +167,36 @@ public class CoordinatorRunStatsTest } @Test - public void testAddWithDebugDimensions() + public void testBuildStatsTable() { - stats.add(DEBUG_STAT_1, 1); - Assert.assertFalse(stats.hasStat(DEBUG_STAT_1)); + stats.add(Stat.ERROR_1, Key.DUTY_1, 10); + stats.add(Stat.INFO_1, Key.DUTY_1, 20); + stats.add(Stat.DEBUG_1, Key.DUTY_1, 30); - stats.add(DEBUG_STAT_1, Key.TIER_1, 1); - Assert.assertFalse(stats.hasStat(DEBUG_STAT_1)); + final String expectedTable + = "\nError: {duty=duty1} ==> {error1=10}" + + "\nInfo : {duty=duty1} ==> {info1=20}" + + "\nDebug: 1 hidden stats. Set 'debugDimensions' to see these." + + "\nTOTAL: 3 stats for 1 dimension keys"; - final CoordinatorRunStats debugStats - = new CoordinatorRunStats(Key.TIER_1.getValues()); - debugStats.add(DEBUG_STAT_1, 1); - Assert.assertFalse(stats.hasStat(DEBUG_STAT_1)); + Assert.assertEquals(expectedTable, stats.buildStatsTable()); + } - debugStats.add(DEBUG_STAT_1, Key.TIER_1, 1); - Assert.assertTrue(debugStats.hasStat(DEBUG_STAT_1)); + @Test + public void testBuildStatsTableWithDebugDimensions() + { + final CoordinatorRunStats debugStats = new CoordinatorRunStats(Key.DUTY_1.getValues()); + debugStats.add(Stat.ERROR_1, Key.DUTY_1, 10); + debugStats.add(Stat.INFO_1, Key.DUTY_1, 20); + debugStats.add(Stat.DEBUG_1, Key.DUTY_1, 30); - debugStats.add(DEBUG_STAT_2, RowKey.of(Dimension.DATASOURCE, "wiki"), 1); - Assert.assertFalse(debugStats.hasStat(DEBUG_STAT_2)); + final String expectedTable + = "\nError: {duty=duty1} ==> {error1=10}" + + "\nInfo : {duty=duty1} ==> {info1=20}" + + "\nDebug: {duty=duty1} ==> {debug1=30}" + + "\nTOTAL: 3 stats for 1 dimension keys"; + + Assert.assertEquals(expectedTable, debugStats.buildStatsTable()); } /** @@ -208,4 +211,15 @@ public class CoordinatorRunStatsTest static final RowKey DUTY_2 = RowKey.of(Dimension.DUTY, "duty2"); } + private static class Stat + { + static final CoordinatorStat ERROR_1 + = CoordinatorStat.toLogAndEmit("error1", "e1", CoordinatorStat.Level.ERROR); + static final CoordinatorStat INFO_1 + = CoordinatorStat.toLogAndEmit("info1", "i1", CoordinatorStat.Level.INFO); + static final CoordinatorStat INFO_2 + = CoordinatorStat.toLogAndEmit("info2", "i2", CoordinatorStat.Level.INFO); + static final CoordinatorStat DEBUG_1 + = CoordinatorStat.toDebugAndEmit("debug1", "d1"); + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java index 39dcc9ce50a..185f8e82630 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java @@ -339,8 +339,6 @@ public class BalanceSegmentsTest CoordinatorDynamicConfig.builder() .withSmartSegmentLoading(false) .withMaxSegmentsToMove(1) - .withUseBatchedSegmentSampler(true) - .withPercentOfSegmentsToConsiderPerMove(40) .build() ) .withBalancerStrategy(balancerStrategy) @@ -355,7 +353,7 @@ public class BalanceSegmentsTest } @Test - public void testUseBatchedSegmentSampler() + public void testMoveForMultipleDatasources() { DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( createHolder(server1, allSegments), @@ -367,7 +365,6 @@ public class BalanceSegmentsTest CoordinatorDynamicConfig.builder() .withSmartSegmentLoading(false) .withMaxSegmentsToMove(2) - .withUseBatchedSegmentSampler(true) .build() ) .withBroadcastDatasources(broadcastDatasources) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java index e64ab0bcc46..c9343da85a6 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java @@ -46,11 +46,9 @@ import java.util.Map; * the simulation. {@link CoordinatorSimulation#stop()} should not be called as * the simulation is stopped when cleaning up after the test in {@link #tearDown()}. *

    - * Tests that verify balancing behaviour should set - * {@link CoordinatorDynamicConfig#useBatchedSegmentSampler()} to true. + * Tests that verify balancing behaviour use batched segment sampling. * Otherwise, the segment sampling is random and can produce repeated values - * leading to flakiness in the tests. The simulation sets this field to true by - * default. + * leading to flakiness in the tests. */ public abstract class CoordinatorSimulationBaseTest implements CoordinatorSimulation.CoordinatorState, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index bd9be366ae7..43f7bd9872a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -151,11 +151,9 @@ public class CoordinatorSimulationBuilder /** * Specifies the CoordinatorDynamicConfig to be used in the simulation. *

    - * Default values: {@code useBatchedSegmentSampler = true}, other params as - * specified in {@link CoordinatorDynamicConfig.Builder}. + * Default values: as specified in {@link CoordinatorDynamicConfig.Builder}. *

    - * Tests that verify balancing behaviour must set - * {@link CoordinatorDynamicConfig#useBatchedSegmentSampler()} to true. + * Tests that verify balancing behaviour use batched segment sampling. * Otherwise, the segment sampling is random and can produce repeated values * leading to flakiness in the tests. The simulation sets this field to true by * default. diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java index 2b81a5c918c..a7744256d5d 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java @@ -46,11 +46,9 @@ public class CoordinatorDynamicConfigTest + " \"mergeBytesLimit\": 1,\n" + " \"mergeSegmentsLimit\" : 1,\n" + " \"maxSegmentsToMove\": 1,\n" - + " \"percentOfSegmentsToConsiderPerMove\": 1,\n" + " \"replicantLifetime\": 1,\n" + " \"replicationThrottleLimit\": 1,\n" + " \"balancerComputeThreads\": 2, \n" - + " \"emitBalancingStats\": true,\n" + " \"killDataSourceWhitelist\": [\"test1\",\"test2\"],\n" + " \"maxSegmentsInNodeLoadingQueue\": 1,\n" + " \"decommissioningNodes\": [\"host1\", \"host2\"],\n" @@ -78,11 +76,8 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - true, - 1, 1, 2, - true, whitelist, false, 1, @@ -101,11 +96,8 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - true, - 1, 1, 2, - true, whitelist, false, 1, @@ -124,11 +116,8 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - true, - 1, 1, 2, - true, whitelist, false, 1, @@ -147,37 +136,8 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - true, - 1, 1, 2, - true, - whitelist, - false, - 1, - ImmutableSet.of("host1"), - 5, - true, - false, - Integer.MAX_VALUE - ); - - actual = CoordinatorDynamicConfig.builder() - .withPercentOfSegmentsToConsiderPerMove(10) - .withUseBatchedSegmentSampler(false) - .build(actual); - assertConfig( - actual, - 1, - 1, - 1, - 1, - 10, - false, - 1, - 1, - 2, - true, whitelist, false, 1, @@ -195,12 +155,9 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - 10, - false, 1, 1, 2, - true, whitelist, false, 1, @@ -218,12 +175,9 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - 10, - false, 1, 1, 2, - true, whitelist, false, 1, @@ -235,6 +189,19 @@ public class CoordinatorDynamicConfigTest ); } + @Test + public void testDeserializationWithUnknownProperties() throws Exception + { + String jsonStr = "{\n" + + " \"unknownProperty\": 2, \n" + + " \"maxSegmentsInNodeLoadingQueue\": 15\n" + + "}\n"; + + CoordinatorDynamicConfig dynamicConfig + = mapper.readValue(jsonStr, CoordinatorDynamicConfig.class); + Assert.assertEquals(15, dynamicConfig.getMaxSegmentsInNodeLoadingQueue()); + } + @Test public void testConstructorWithNullsShouldKillUnusedSegmentsInAllDataSources() { @@ -243,12 +210,9 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - null, - false, 1, 2, 10, - true, null, null, null, @@ -273,12 +237,9 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - null, - false, 1, 2, 10, - true, ImmutableSet.of("test1"), null, null, @@ -306,7 +267,6 @@ public class CoordinatorDynamicConfigTest + " \"replicantLifetime\": 1,\n" + " \"replicationThrottleLimit\": 1,\n" + " \"balancerComputeThreads\": 2, \n" - + " \"emitBalancingStats\": true,\n" + " \"killDataSourceWhitelist\": [\"test1\",\"test2\"],\n" + " \"maxSegmentsInNodeLoadingQueue\": 1\n" + "}\n"; @@ -328,11 +288,9 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - 100, - true, 1, + 1, 1, 2, - true, whitelist, false, 1, @@ -350,12 +308,9 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - 100, - true, 1, 1, 2, - true, whitelist, false, 1, @@ -373,12 +328,9 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - 100, - true, 1, 1, 2, - true, whitelist, false, 1, @@ -391,18 +343,16 @@ public class CoordinatorDynamicConfigTest } @Test - public void testSerdeWithStringinKillDataSourceWhitelist() throws Exception + public void testSerdeWithStringInKillDataSourceWhitelist() throws Exception { String jsonStr = "{\n" + " \"millisToWaitBeforeDeleting\": 1,\n" + " \"mergeBytesLimit\": 1,\n" + " \"mergeSegmentsLimit\" : 1,\n" + " \"maxSegmentsToMove\": 1,\n" - + " \"percentOfSegmentsToConsiderPerMove\": 1,\n" + " \"replicantLifetime\": 1,\n" + " \"replicationThrottleLimit\": 1,\n" + " \"balancerComputeThreads\": 2, \n" - + " \"emitBalancingStats\": true,\n" + " \"killDataSourceWhitelist\": \"test1, test2\", \n" + " \"maxSegmentsInNodeLoadingQueue\": 1\n" + "}\n"; @@ -423,11 +373,8 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - true, - 1, 1, 2, - true, ImmutableSet.of("test1", "test2"), false, 1, @@ -440,19 +387,15 @@ public class CoordinatorDynamicConfigTest } @Test - public void testSerdeHandleInvalidPercentOfSegmentsToConsiderPerMove() + public void testSerdeHandlesInvalidDecommissioningPercentToMove() { - final String errorMsg = "'percentOfSegmentsToConsiderPerMove' should be between 1 and 100"; + final String errorMsg = "'decommissioningMaxPercentOfMaxSegmentsToMove' should be in range [0, 100]"; assertThatDeserializationFailsWithMessage( - "{\"percentOfSegmentsToConsiderPerMove\": 0}", + "{\"decommissioningMaxPercentOfMaxSegmentsToMove\": -1}", errorMsg ); assertThatDeserializationFailsWithMessage( - "{\"percentOfSegmentsToConsiderPerMove\": -100}", - errorMsg - ); - assertThatDeserializationFailsWithMessage( - "{\"percentOfSegmentsToConsiderPerMove\": 105}", + "{\"decommissioningMaxPercentOfMaxSegmentsToMove\": 105}", errorMsg ); } @@ -468,7 +411,6 @@ public class CoordinatorDynamicConfigTest + " \"replicantLifetime\": 1,\n" + " \"replicationThrottleLimit\": 1,\n" + " \"balancerComputeThreads\": 2, \n" - + " \"emitBalancingStats\": true,\n" + " \"killDataSourceWhitelist\": [\"test1\",\"test2\"],\n" + " \"maxSegmentsInNodeLoadingQueue\": 1,\n" + " \"decommissioningNodes\": [\"host1\", \"host2\"],\n" @@ -489,12 +431,9 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - 100, - true, 1, 1, 2, - true, whitelist, false, 1, @@ -514,11 +453,9 @@ public class CoordinatorDynamicConfigTest + " \"mergeBytesLimit\": 1,\n" + " \"mergeSegmentsLimit\" : 1,\n" + " \"maxSegmentsToMove\": 1,\n" - + " \"percentOfSegmentsToConsiderPerMove\": 1,\n" + " \"replicantLifetime\": 1,\n" + " \"replicationThrottleLimit\": 1,\n" + " \"balancerComputeThreads\": 2, \n" - + " \"emitBalancingStats\": true,\n" + " \"killAllDataSources\": true,\n" + " \"maxSegmentsInNodeLoadingQueue\": 1\n" + "}\n"; @@ -537,11 +474,8 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - true, - 1, 1, 2, - true, ImmutableSet.of(), true, 1, @@ -557,8 +491,7 @@ public class CoordinatorDynamicConfigTest // so this is a valid config jsonStr = "{\n" + " \"killDataSourceWhitelist\": [\"test1\",\"test2\"],\n" - + " \"killAllDataSources\": true,\n" - + " \"percentOfSegmentsToConsiderPerMove\": 1\n" + + " \"killAllDataSources\": true\n" + "}\n"; actual = mapper.readValue(jsonStr, CoordinatorDynamicConfig.class); @@ -574,11 +507,9 @@ public class CoordinatorDynamicConfigTest + " \"mergeBytesLimit\": 1,\n" + " \"mergeSegmentsLimit\" : 1,\n" + " \"maxSegmentsToMove\": 1,\n" - + " \"percentOfSegmentsToConsiderPerMove\": 1,\n" + " \"replicantLifetime\": 1,\n" + " \"replicationThrottleLimit\": 1,\n" + " \"balancerComputeThreads\": 2, \n" - + " \"emitBalancingStats\": true,\n" + " \"killAllDataSources\": true\n" + "}\n"; @@ -596,11 +527,8 @@ public class CoordinatorDynamicConfigTest 1, 1, 1, - true, - 1, 1, 2, - true, ImmutableSet.of(), true, EXPECTED_DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE, @@ -623,12 +551,9 @@ public class CoordinatorDynamicConfigTest 524288000, 100, 100, - 100, - true, 15, 500, 1, - false, emptyList, true, EXPECTED_DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE, @@ -654,12 +579,9 @@ public class CoordinatorDynamicConfigTest 524288000, 100, 100, - 100, - true, 15, 500, 1, - false, ImmutableSet.of("DATASOURCE"), false, EXPECTED_DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE, @@ -699,9 +621,6 @@ public class CoordinatorDynamicConfigTest null, null, null, - null, - null, - null, null ).build(current) ); @@ -733,7 +652,7 @@ public class CoordinatorDynamicConfigTest } @Test - public void testEqualsAndHashCodeSanity() + public void testEqualsAndHashCode() { CoordinatorDynamicConfig config1 = CoordinatorDynamicConfig.builder().build(); CoordinatorDynamicConfig config2 = CoordinatorDynamicConfig.builder().build(); @@ -747,12 +666,9 @@ public class CoordinatorDynamicConfigTest long expectedMergeBytesLimit, int expectedMergeSegmentsLimit, int expectedMaxSegmentsToMove, - int expectedPercentOfSegmentsToConsiderPerMove, - boolean expectedUseBatchedSegmentSampler, int expectedReplicantLifetime, int expectedReplicationThrottleLimit, int expectedBalancerComputeThreads, - boolean expectedEmitingBalancingStats, Set expectedSpecificDataSourcesToKillUnusedSegmentsIn, boolean expectedKillUnusedSegmentsInAllDataSources, int expectedMaxSegmentsInNodeLoadingQueue, @@ -770,12 +686,9 @@ public class CoordinatorDynamicConfigTest Assert.assertEquals(expectedMergeBytesLimit, config.getMergeBytesLimit()); Assert.assertEquals(expectedMergeSegmentsLimit, config.getMergeSegmentsLimit()); Assert.assertEquals(expectedMaxSegmentsToMove, config.getMaxSegmentsToMove()); - Assert.assertEquals(expectedPercentOfSegmentsToConsiderPerMove, config.getPercentOfSegmentsToConsiderPerMove(), 0); - Assert.assertEquals(expectedUseBatchedSegmentSampler, config.useBatchedSegmentSampler()); Assert.assertEquals(expectedReplicantLifetime, config.getReplicantLifetime()); Assert.assertEquals(expectedReplicationThrottleLimit, config.getReplicationThrottleLimit()); Assert.assertEquals(expectedBalancerComputeThreads, config.getBalancerComputeThreads()); - Assert.assertEquals(expectedEmitingBalancingStats, config.emitBalancingStats()); Assert.assertEquals( expectedSpecificDataSourcesToKillUnusedSegmentsIn, config.getSpecificDataSourcesToKillUnusedSegmentsIn() diff --git a/web-console/src/druid-models/coordinator-dynamic-config/coordinator-dynamic-config.tsx b/web-console/src/druid-models/coordinator-dynamic-config/coordinator-dynamic-config.tsx index f1d460a3414..f8ca99c221b 100644 --- a/web-console/src/druid-models/coordinator-dynamic-config/coordinator-dynamic-config.tsx +++ b/web-console/src/druid-models/coordinator-dynamic-config/coordinator-dynamic-config.tsx @@ -24,7 +24,6 @@ import type { Field } from '../../components'; export interface CoordinatorDynamicConfig { maxSegmentsToMove?: number; balancerComputeThreads?: number; - emitBalancingStats?: boolean; killAllDataSources?: boolean; killDataSourceWhitelist?: string[]; killPendingSegmentsSkipList?: string[]; @@ -58,17 +57,6 @@ export const COORDINATOR_DYNAMIC_CONFIG_FIELDS: Field[ ), }, - { - name: 'emitBalancingStats', - type: 'boolean', - defaultValue: false, - info: ( - <> - Boolean flag for whether or not we should emit balancing stats. This is an expensive - operation. - - ), - }, { name: 'killDataSourceWhitelist', label: 'Kill datasource whitelist', @@ -183,19 +171,6 @@ export const COORDINATOR_DYNAMIC_CONFIG_FIELDS: Field[ ), }, - { - name: 'useBatchedSegmentSampler', - type: 'boolean', - defaultValue: true, - info: ( - <> - Boolean flag for whether or not we should use the Reservoir Sampling with a reservoir of - size k instead of fixed size 1 to pick segments to move. This option can be enabled to speed - up segment balancing process, especially if there are huge number of segments in the cluster - or if there are too many segments to move. - - ), - }, { name: 'useRoundRobinSegmentAssignment', type: 'boolean', @@ -208,28 +183,6 @@ export const COORDINATOR_DYNAMIC_CONFIG_FIELDS: Field[ ), }, - { - name: 'percentOfSegmentsToConsiderPerMove', - type: 'number', - defaultValue: 100, - info: ( - <> - Deprecated. This will eventually be phased out by the batched segment sampler. You can - enable the batched segment sampler now by setting the dynamic Coordinator config, - useBatchedSegmentSampler, to true. Note that if you choose to enable the batched segment - sampler, percentOfSegmentsToConsiderPerMove will no longer have any effect on balancing. If - useBatchedSegmentSampler == false, this config defines the percentage of the total number of - segments in the cluster that are considered every time a segment needs to be selected for a - move. Druid orders servers by available capacity ascending (the least available capacity - first) and then iterates over the servers. For each server, Druid iterates over the segments - on the server, considering them for moving. The default config of 100% means that every - segment on every server is a candidate to be moved. This should make sense for most small to - medium-sized clusters. However, an admin may find it preferable to drop this value lower if - they don't think that it is worthwhile to consider every single segment in the cluster - each time it is looking for a segment to move. - - ), - }, { name: 'pauseCoordination', type: 'boolean', From 87bb1b970915631149812e7a92892bc8850cf3eb Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 6 Jul 2023 13:04:53 +0530 Subject: [PATCH 69/74] Fix bug during initialization of HttpServerInventoryView (#14517) If a server is removed during `HttpServerInventoryView.serverInventoryInitialized`, the initialization gets stuck as this server is never synced. The method eventually times out (default 250s). Fix: Mark a server as stopped if it is removed. `serverInventoryInitialized` only waits for non-stopped servers to sync. Other changes: - Add new metrics for better debugging of slow broker/coordinator startup - `segment/serverview/sync/healthy`: whether the server view is syncing properly with a server - `segment/serverview/sync/unstableTime`: time for which sync with a server has been unstable - Clean up logging in `HttpServerInventoryView` and `ChangeRequestHttpSyncer` - Minor refactor for readability - Add utility class `Stopwatch` - Add tests and stubs --- docs/operations/metrics.md | 5 +- .../overlord/hrtr/HttpRemoteTaskRunner.java | 2 +- .../indexing/overlord/hrtr/WorkerHolder.java | 5 +- .../hrtr/HttpRemoteTaskRunnerTest.java | 2 +- .../overlord/hrtr/WorkerHolderTest.java | 1 + .../druid/java/util/common/Stopwatch.java | 111 +++ .../druid/java/util/common/StopwatchTest.java | 70 ++ .../java/util/metrics/StubServiceEmitter.java | 10 + .../org/apache/druid/client/DruidServer.java | 13 + ...lteredHttpServerInventoryViewProvider.java | 12 + .../druid/client/HttpServerInventoryView.java | 191 +++-- .../HttpServerInventoryViewProvider.java | 12 + .../druid/discovery/DiscoveryDruidNode.java | 24 + .../coordination/ChangeRequestHttpSyncer.java | 219 ++--- .../client/HttpServerInventoryViewTest.java | 783 +++++++++++------- .../client/TestChangeRequestHttpClient.java | 164 ++++ 16 files changed, 1132 insertions(+), 492 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/java/util/common/Stopwatch.java create mode 100644 processing/src/test/java/org/apache/druid/java/util/common/StopwatchTest.java create mode 100644 server/src/test/java/org/apache/druid/client/TestChangeRequestHttpClient.java diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 9003d0cb24f..b742fff795d 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -68,7 +68,8 @@ Metrics may have additional dimensions beyond those listed above. |`init/metadatacache/time`|Time taken to initialize the broker segment metadata cache. Useful to detect if brokers are taking too long to start||Depends on the number of segments.| |`segment/metadatacache/refresh/count`|Number of segments to refresh in broker segment metadata cache.|`dataSource`| |`segment/metadatacache/refresh/time`|Time taken to refresh segments in broker segment metadata cache.|`dataSource`| - +|`segment/serverview/sync/healthy`|Sync status of the Broker with a segment-loading server such as a Historical or Peon. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled. This metric can be used in conjunction with `segment/serverview/sync/unstableTime` to debug slow startup of Brokers.|`server`, `tier`|1 for fully synced servers, 0 otherwise| +|`segment/serverview/sync/unstableTime`|Time in milliseconds for which the Broker has been failing to sync with a segment-loading server. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled.|`server`, `tier`|Not emitted for synced servers.| ### Historical @@ -324,6 +325,8 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina |`metadata/kill/rule/count`|Total number of rules that were automatically deleted from metadata store per each Coordinator kill rule duty run. This metric can help adjust `druid.coordinator.kill.rule.durationToRetain` configuration based on whether more or less rules need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.rule.on` is set to true.| |Varies| |`metadata/kill/datasource/count`|Total number of datasource metadata that were automatically deleted from metadata store per each Coordinator kill datasource duty run (Note: datasource metadata only exists for datasource created from supervisor). This metric can help adjust `druid.coordinator.kill.datasource.durationToRetain` configuration based on whether more or less datasource metadata need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.datasource.on` is set to true.| |Varies| |`init/serverview/time`|Time taken to initialize the coordinator server view.||Depends on the number of segments| +|`segment/serverview/sync/healthy`|Sync status of the Coordinator with a segment-loading server such as a Historical or Peon. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled. This metric can be used in conjunction with `segment/serverview/sync/unstableTime` to debug slow startup of the Coordinator.|`server`, `tier`|1 for fully synced servers, 0 otherwise| +|`segment/serverview/sync/unstableTime`|Time in milliseconds for which the Coordinator has been failing to sync with a segment-loading server. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled.|`server`, `tier`|Not emitted for synced servers.| ## General Health diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java index 8674d5a1113..a6354c90628 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java @@ -791,7 +791,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer final Set> workerEntrySet = ImmutableSet.copyOf(workers.entrySet()); for (Map.Entry e : workerEntrySet) { WorkerHolder workerHolder = e.getValue(); - if (!workerHolder.getUnderlyingSyncer().isOK()) { + if (workerHolder.getUnderlyingSyncer().needsReset()) { synchronized (workers) { // check again that server is still there and only then reset. if (workers.containsKey(e.getKey())) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java index 0bf4de0df86..a8fc5306048 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolder.java @@ -55,7 +55,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -340,7 +339,7 @@ public class WorkerHolder public void waitForInitialization() throws InterruptedException { - if (!syncer.awaitInitialization(3 * syncer.getServerHttpTimeout(), TimeUnit.MILLISECONDS)) { + if (!syncer.awaitInitialization()) { throw new RE("Failed to sync with worker[%s].", worker.getHost()); } } @@ -348,7 +347,7 @@ public class WorkerHolder public boolean isInitialized() { try { - return syncer.awaitInitialization(1, TimeUnit.MILLISECONDS); + return syncer.isInitialized(); } catch (InterruptedException ignored) { Thread.currentThread().interrupt(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java index fe1b0ca498f..6cc6329c584 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java @@ -1789,7 +1789,7 @@ public class HttpRemoteTaskRunnerTest private WorkerHolder createNonSyncingWorkerHolder(Worker worker) { ChangeRequestHttpSyncer syncer = EasyMock.createMock(ChangeRequestHttpSyncer.class); - EasyMock.expect(syncer.isOK()).andReturn(false).anyTimes(); + EasyMock.expect(syncer.needsReset()).andReturn(true).anyTimes(); EasyMock.expect(syncer.getDebugInfo()).andReturn(Collections.emptyMap()).anyTimes(); WorkerHolder workerHolder = EasyMock.createMock(WorkerHolder.class); EasyMock.expect(workerHolder.getUnderlyingSyncer()).andReturn(syncer).anyTimes(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolderTest.java index 5f4a9b843a6..f13a0b0cce8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/WorkerHolderTest.java @@ -78,6 +78,7 @@ public class WorkerHolderTest ChangeRequestHttpSyncer.Listener syncListener = workerHolder.createSyncListener(); Assert.assertTrue(workerHolder.disabled.get()); + Assert.assertFalse(workerHolder.isInitialized()); syncListener.fullSync( ImmutableList.of( diff --git a/processing/src/main/java/org/apache/druid/java/util/common/Stopwatch.java b/processing/src/main/java/org/apache/druid/java/util/common/Stopwatch.java new file mode 100644 index 00000000000..2d941828a0e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/common/Stopwatch.java @@ -0,0 +1,111 @@ +/* + * 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.druid.java.util.common; + +import com.google.common.base.Ticker; +import org.joda.time.Duration; + +import java.util.concurrent.TimeUnit; + +/** + * Thread-safe wrapper over {@link com.google.common.base.Stopwatch}. + *

    + * Thread safety has been limited to the start/stop methods for now as they are + * the only ones that can throw an exception in an illegal state and are thus + * vulnerable to race conditions. + */ +public class Stopwatch +{ + private final com.google.common.base.Stopwatch delegate; + + public static Stopwatch createStarted() + { + return new Stopwatch(com.google.common.base.Stopwatch.createStarted()); + } + + public static Stopwatch createUnstarted() + { + return new Stopwatch(com.google.common.base.Stopwatch.createUnstarted()); + } + + public static Stopwatch createStarted(Ticker ticker) + { + return new Stopwatch(com.google.common.base.Stopwatch.createStarted(ticker)); + } + + private Stopwatch(com.google.common.base.Stopwatch delegate) + { + this.delegate = delegate; + } + + public synchronized void start() + { + delegate.start(); + } + + public synchronized void stop() + { + delegate.stop(); + } + + public synchronized void reset() + { + delegate.reset(); + } + + /** + * Invokes {@code reset().start()} on the underlying {@link com.google.common.base.Stopwatch}. + */ + public synchronized void restart() + { + delegate.reset().start(); + } + + public synchronized boolean isRunning() + { + return delegate.isRunning(); + } + + /** + * Returns the milliseconds elapsed on the stopwatch. + */ + public long millisElapsed() + { + return delegate.elapsed(TimeUnit.MILLISECONDS); + } + + /** + * Checks if the given duration has already elapsed on the stopwatch. + */ + public boolean hasElapsed(Duration duration) + { + return millisElapsed() >= duration.getMillis(); + } + + /** + * Checks that the given duration has not elapsed on the stopwatch. Calling this + * method is the same as {@code !stopwatch.hasElapsed(duration)}. + */ + public boolean hasNotElapsed(Duration duration) + { + return !hasElapsed(duration); + } + +} diff --git a/processing/src/test/java/org/apache/druid/java/util/common/StopwatchTest.java b/processing/src/test/java/org/apache/druid/java/util/common/StopwatchTest.java new file mode 100644 index 00000000000..06bed222e00 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/java/util/common/StopwatchTest.java @@ -0,0 +1,70 @@ +/* + * 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.druid.java.util.common; + +import com.google.common.testing.FakeTicker; +import org.joda.time.Duration; +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.TimeUnit; + +public class StopwatchTest +{ + + @Test + public void testDuplicateStartThrowsException() + { + Stopwatch stopwatch = Stopwatch.createStarted(); + Assert.assertThrows(IllegalStateException.class, stopwatch::start); + } + + @Test + public void testDuplicateStopThrowsException() + { + Stopwatch stopwatch = Stopwatch.createUnstarted(); + Assert.assertThrows(IllegalStateException.class, stopwatch::stop); + } + + @Test + public void testMillisElapsed() + { + FakeTicker fakeTicker = new FakeTicker(); + Stopwatch stopwatch = Stopwatch.createStarted(fakeTicker); + fakeTicker.advance(100, TimeUnit.MILLISECONDS); + stopwatch.stop(); + + Assert.assertEquals(100, stopwatch.millisElapsed()); + } + + @Test + public void testHasElapsed() + { + FakeTicker fakeTicker = new FakeTicker(); + Stopwatch stopwatch = Stopwatch.createStarted(fakeTicker); + fakeTicker.advance(100, TimeUnit.MILLISECONDS); + stopwatch.stop(); + + Assert.assertTrue(stopwatch.hasElapsed(Duration.millis(50))); + Assert.assertTrue(stopwatch.hasElapsed(Duration.millis(100))); + Assert.assertTrue(stopwatch.hasNotElapsed(Duration.millis(101))); + Assert.assertTrue(stopwatch.hasNotElapsed(Duration.millis(500))); + } +} diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java b/processing/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java index 39524581579..52049c6956f 100644 --- a/processing/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java +++ b/processing/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java @@ -20,6 +20,7 @@ package org.apache.druid.java.util.metrics; import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; @@ -32,6 +33,7 @@ import java.util.Map; public class StubServiceEmitter extends ServiceEmitter implements MetricsVerifier { private final List events = new ArrayList<>(); + private final List alertEvents = new ArrayList<>(); private final Map> metricEvents = new HashMap<>(); public StubServiceEmitter(String service, String host) @@ -46,6 +48,8 @@ public class StubServiceEmitter extends ServiceEmitter implements MetricsVerifie ServiceMetricEvent metricEvent = (ServiceMetricEvent) event; metricEvents.computeIfAbsent(metricEvent.getMetric(), name -> new ArrayList<>()) .add(metricEvent); + } else if (event instanceof AlertEvent) { + alertEvents.add((AlertEvent) event); } events.add(event); } @@ -58,6 +62,11 @@ public class StubServiceEmitter extends ServiceEmitter implements MetricsVerifie return events; } + public List getAlerts() + { + return alertEvents; + } + @Override public List getMetricValues( String metricName, @@ -92,6 +101,7 @@ public class StubServiceEmitter extends ServiceEmitter implements MetricsVerifie public void flush() { events.clear(); + alertEvents.clear(); metricEvents.clear(); } diff --git a/server/src/main/java/org/apache/druid/client/DruidServer.java b/server/src/main/java/org/apache/druid/client/DruidServer.java index 6c52866d058..59010448793 100644 --- a/server/src/main/java/org/apache/druid/client/DruidServer.java +++ b/server/src/main/java/org/apache/druid/client/DruidServer.java @@ -320,4 +320,17 @@ public class DruidServer implements Comparable immutableDataSources.values().stream().mapToInt(dataSource -> dataSource.getSegments().size()).sum(); return new ImmutableDruidServer(metadata, size, immutableDataSources, totalSegments); } + + public DruidServer copyWithoutSegments() + { + return new DruidServer( + getName(), + getHostAndPort(), + getHostAndTlsPort(), + getMaxSize(), + getType(), + getTier(), + getPriority() + ); + } } diff --git a/server/src/main/java/org/apache/druid/client/FilteredHttpServerInventoryViewProvider.java b/server/src/main/java/org/apache/druid/client/FilteredHttpServerInventoryViewProvider.java index bce9c562f6b..b6c3ba2a63f 100644 --- a/server/src/main/java/org/apache/druid/client/FilteredHttpServerInventoryViewProvider.java +++ b/server/src/main/java/org/apache/druid/client/FilteredHttpServerInventoryViewProvider.java @@ -25,6 +25,8 @@ import com.google.common.base.Predicates; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.guice.annotations.EscalatedClient; import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; import javax.validation.constraints.NotNull; @@ -51,6 +53,14 @@ public class FilteredHttpServerInventoryViewProvider implements FilteredServerIn @NotNull private DruidNodeDiscoveryProvider druidNodeDiscoveryProvider; + @JacksonInject + @NotNull + private ScheduledExecutorFactory executorFactory; + + @JacksonInject + @NotNull + private ServiceEmitter serviceEmitter; + @Override public HttpServerInventoryView get() { @@ -60,6 +70,8 @@ public class FilteredHttpServerInventoryViewProvider implements FilteredServerIn druidNodeDiscoveryProvider, Predicates.alwaysFalse(), config, + serviceEmitter, + executorFactory, "FilteredHttpServerInventoryView" ); } diff --git a/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java b/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java index 2f86c46064b..ebc67527941 100644 --- a/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java +++ b/server/src/main/java/org/apache/druid/client/HttpServerInventoryView.java @@ -27,6 +27,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Predicates; import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import com.google.common.net.HostAndPort; @@ -39,10 +40,13 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.ChangeRequestHttpSyncer; @@ -54,12 +58,12 @@ import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; +import org.joda.time.Duration; import java.net.MalformedURLException; import java.net.URL; import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -107,11 +111,14 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer private final ConcurrentHashMap servers = new ConcurrentHashMap<>(); private final String execNamePrefix; - private volatile ScheduledExecutorService executor; + private final ScheduledExecutorFactory executorFactory; + private volatile ScheduledExecutorService inventorySyncExecutor; + private volatile ScheduledExecutorService monitoringExecutor; private final HttpClient httpClient; private final ObjectMapper smileMapper; private final HttpServerInventoryViewConfig config; + private final ServiceEmitter serviceEmitter; public HttpServerInventoryView( final ObjectMapper smileMapper, @@ -119,6 +126,8 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, final Predicate> defaultFilter, final HttpServerInventoryViewConfig config, + final ServiceEmitter serviceEmitter, + final ScheduledExecutorFactory executorFactory, final String execNamePrefix ) { @@ -128,6 +137,8 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer this.defaultFilter = defaultFilter; this.finalPredicate = defaultFilter; this.config = config; + this.serviceEmitter = serviceEmitter; + this.executorFactory = executorFactory; this.execNamePrefix = execNamePrefix; } @@ -137,16 +148,17 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer { synchronized (lifecycleLock) { if (!lifecycleLock.canStart()) { - throw new ISE("can't start."); + throw new ISE("Could not start lifecycle"); } - log.info("Starting %s.", execNamePrefix); + log.info("Starting executor[%s].", execNamePrefix); try { - executor = ScheduledExecutors.fixed( + inventorySyncExecutor = executorFactory.create( config.getNumThreads(), execNamePrefix + "-%s" ); + monitoringExecutor = executorFactory.create(1, execNamePrefix + "-monitor-%s"); DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForService(DataNodeService.DISCOVERY_SERVICE_KEY); druidNodeDiscovery.registerListener( @@ -170,7 +182,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer public void nodeViewInitialized() { if (!initialized.getAndSet(true)) { - executor.execute(HttpServerInventoryView.this::serverInventoryInitialized); + inventorySyncExecutor.execute(HttpServerInventoryView.this::serverInventoryInitialized); } } @@ -204,7 +216,18 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer } ); - scheduleSyncMonitoring(); + ScheduledExecutors.scheduleWithFixedDelay( + monitoringExecutor, + Duration.standardSeconds(60), + Duration.standardMinutes(5), + this::checkAndResetUnhealthyServers + ); + ScheduledExecutors.scheduleAtFixedRate( + monitoringExecutor, + Duration.standardSeconds(30), + Duration.standardMinutes(1), + this::emitServerStatusMetrics + ); lifecycleLock.started(); } @@ -212,7 +235,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer lifecycleLock.exitStart(); } - log.info("Started %s.", execNamePrefix); + log.info("Started executor[%s].", execNamePrefix); } } @@ -224,13 +247,16 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer throw new ISE("can't stop."); } - log.info("Stopping %s.", execNamePrefix); + log.info("Stopping executor[%s].", execNamePrefix); - if (executor != null) { - executor.shutdownNow(); + if (inventorySyncExecutor != null) { + inventorySyncExecutor.shutdownNow(); + } + if (monitoringExecutor != null) { + monitoringExecutor.shutdownNow(); } - log.info("Stopped %s.", execNamePrefix); + log.info("Stopped executor[%s].", execNamePrefix); } } @@ -249,10 +275,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer segmentCallbacks.put(filteringSegmentCallback, exec); segmentPredicates.put(filteringSegmentCallback, filter); - finalPredicate = Predicates.or( - defaultFilter, - Predicates.or(segmentPredicates.values()) - ); + updateFinalPredicate(); } @Override @@ -308,10 +331,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) { segmentCallbacks.remove(entry.getKey()); if (segmentPredicates.remove(entry.getKey()) != null) { - finalPredicate = Predicates.or( - defaultFilter, - Predicates.or(segmentPredicates.values()) - ); + updateFinalPredicate(); } } } @@ -320,7 +340,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer } } - private void runServerCallbacks(final DruidServer server) + private void runServerRemovedCallbacks(final DruidServer server) { for (final Map.Entry entry : serverCallbacks.entrySet()) { entry.getValue().execute( @@ -338,8 +358,11 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer } } - //best effort wait for first segment listing fetch from all servers and then call - //segmentViewInitialized on all registered segment callbacks. + /** + * Waits until the sync wait timeout for all servers to be synced at least once. + * Finally calls {@link SegmentCallback#segmentViewInitialized()} regardless of + * whether all servers synced successfully or not. + */ private void serverInventoryInitialized() { long start = System.currentTimeMillis(); @@ -360,13 +383,11 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer throw new RE(ex, "Interrupted while waiting for queryable server initial successful sync."); } - log.info("Checking whether all servers have been synced at least once yet...."); - Iterator iter = uninitializedServers.iterator(); - while (iter.hasNext()) { - if (iter.next().isSyncedSuccessfullyAtleastOnce()) { - iter.remove(); - } - } + log.info("Waiting for [%d] servers to sync successfully.", uninitializedServers.size()); + uninitializedServers.removeIf( + serverHolder -> serverHolder.isSyncedSuccessfullyAtleastOnce() + || serverHolder.isStopped() + ); } if (uninitializedServers.isEmpty()) { @@ -380,18 +401,13 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer } } - log.info("Calling SegmentCallback.segmentViewInitialized() for all callbacks."); + log.info("Invoking segment view initialized callbacks."); + runSegmentCallbacks(SegmentCallback::segmentViewInitialized); + } - runSegmentCallbacks( - new Function() - { - @Override - public CallbackAction apply(SegmentCallback input) - { - return input.segmentViewInitialized(); - } - } - ); + private void updateFinalPredicate() + { + finalPredicate = Predicates.or(defaultFilter, Predicates.or(segmentPredicates.values())); } @VisibleForTesting @@ -417,9 +433,9 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer if (holder != null) { log.info("Server[%s] disappeared.", server.getName()); holder.stop(); - runServerCallbacks(holder.druidServer); + runServerRemovedCallbacks(holder.druidServer); } else { - log.info("Server[%s] did not exist. Removal notification ignored.", server.getName()); + log.info("Ignoring remove notification for unknown server[%s].", server.getName()); } } } @@ -443,61 +459,57 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer return result; } - private void scheduleSyncMonitoring() - { - executor.scheduleAtFixedRate( - () -> { - log.debug("Running the Sync Monitoring."); - - try { - syncMonitoring(); - } - catch (Exception ex) { - if (ex instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } else { - log.makeAlert(ex, "Exception in sync monitoring.").emit(); - } - } - }, - 1, - 5, - TimeUnit.MINUTES - ); - } - @VisibleForTesting - void syncMonitoring() + void checkAndResetUnhealthyServers() { // Ensure that the collection is not being modified during iteration. Iterate over a copy final Set> serverEntrySet = ImmutableSet.copyOf(servers.entrySet()); for (Map.Entry e : serverEntrySet) { DruidServerHolder serverHolder = e.getValue(); - if (!serverHolder.syncer.isOK()) { + if (serverHolder.syncer.needsReset()) { synchronized (servers) { - // check again that server is still there and only then reset. + // Reset only if the server is still present in the map if (servers.containsKey(e.getKey())) { - log.makeAlert( - "Server[%s] is not syncing properly. Current state is [%s]. Resetting it.", + log.warn( + "Resetting server[%s] with state[%s] as it is not syncing properly.", serverHolder.druidServer.getName(), serverHolder.syncer.getDebugInfo() - ).emit(); + ); serverRemoved(serverHolder.druidServer); - serverAdded(new DruidServer( - serverHolder.druidServer.getName(), - serverHolder.druidServer.getHostAndPort(), - serverHolder.druidServer.getHostAndTlsPort(), - serverHolder.druidServer.getMaxSize(), - serverHolder.druidServer.getType(), - serverHolder.druidServer.getTier(), - serverHolder.druidServer.getPriority() - )); + serverAdded(serverHolder.druidServer.copyWithoutSegments()); } } } } } + private void emitServerStatusMetrics() + { + final ServiceMetricEvent.Builder eventBuilder = ServiceMetricEvent.builder(); + try { + final Map serversCopy = ImmutableMap.copyOf(servers); + serversCopy.forEach((serverName, serverHolder) -> { + final DruidServer server = serverHolder.druidServer; + eventBuilder.setDimension("tier", server.getTier()); + eventBuilder.setDimension("server", serverName); + + final boolean isSynced = serverHolder.syncer.isSyncedSuccessfully(); + serviceEmitter.emit( + eventBuilder.build("segment/serverview/sync/healthy", isSynced ? 1 : 0) + ); + final long unstableTimeMillis = serverHolder.syncer.getUnstableTimeMillis(); + if (unstableTimeMillis > 0) { + serviceEmitter.emit( + eventBuilder.build("segment/serverview/sync/unstableTime", unstableTimeMillis) + ); + } + }); + } + catch (Exception e) { + log.error(e, "Error while emitting server status metrics"); + } + } + @Override public boolean isStarted() { @@ -514,6 +526,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer private class DruidServerHolder { private final DruidServer druidServer; + private final AtomicBoolean stopped = new AtomicBoolean(false); private final ChangeRequestHttpSyncer syncer; @@ -526,7 +539,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer this.syncer = new ChangeRequestHttpSyncer<>( smileMapper, httpClient, - executor, + inventorySyncExecutor, new URL(druidServer.getScheme(), hostAndPort.getHostText(), hostAndPort.getPort(), "/"), "/druid-internal/v1/segments", SEGMENT_LIST_RESP_TYPE_REF, @@ -548,19 +561,21 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer void stop() { syncer.stop(); + stopped.set(true); + } + + boolean isStopped() + { + return stopped.get(); } boolean isSyncedSuccessfullyAtleastOnce() { try { - return syncer.awaitInitialization(1, TimeUnit.MILLISECONDS); + return syncer.isInitialized(); } catch (InterruptedException ex) { - throw new RE( - ex, - "Interrupted while waiting for queryable server[%s] initial successful sync.", - druidServer.getName() - ); + throw new ISE(ex, "Interrupted while waiting for first sync with server[%s].", druidServer.getName()); } } diff --git a/server/src/main/java/org/apache/druid/client/HttpServerInventoryViewProvider.java b/server/src/main/java/org/apache/druid/client/HttpServerInventoryViewProvider.java index 7f008c58798..df939c406c7 100644 --- a/server/src/main/java/org/apache/druid/client/HttpServerInventoryViewProvider.java +++ b/server/src/main/java/org/apache/druid/client/HttpServerInventoryViewProvider.java @@ -25,6 +25,8 @@ import com.google.common.base.Predicates; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.guice.annotations.EscalatedClient; import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; import javax.validation.constraints.NotNull; @@ -51,6 +53,14 @@ public class HttpServerInventoryViewProvider implements ServerInventoryViewProvi @NotNull private DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = null; + @JacksonInject + @NotNull + private ScheduledExecutorFactory executorFactory; + + @JacksonInject + @NotNull + private ServiceEmitter serviceEmitter = null; + @Override public HttpServerInventoryView get() { @@ -60,6 +70,8 @@ public class HttpServerInventoryViewProvider implements ServerInventoryViewProvi druidNodeDiscoveryProvider, Predicates.alwaysTrue(), config, + serviceEmitter, + executorFactory, "HttpServerInventoryView" ); } diff --git a/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java b/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java index 461e5fea0df..3a62e5344b1 100644 --- a/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java +++ b/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; +import org.apache.druid.client.DruidServer; import org.apache.druid.jackson.StringObjectPairList; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; @@ -199,6 +200,29 @@ public class DiscoveryDruidNode return null; } + public DruidServer toDruidServer() + { + final DataNodeService dataNodeService = getService( + DataNodeService.DISCOVERY_SERVICE_KEY, + DataNodeService.class + ); + + final DruidNode druidNode = getDruidNode(); + if (dataNodeService == null || druidNode == null) { + return null; + } + + return new DruidServer( + druidNode.getHostAndPortToUse(), + druidNode.getHostAndPort(), + druidNode.getHostAndTlsPort(), + dataNodeService.getMaxSize(), + dataNodeService.getServerType(), + dataNodeService.getTier(), + dataNodeService.getPriority() + ); + } + @Override public boolean equals(Object o) { diff --git a/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncer.java b/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncer.java index 8076f3eddc1..8af89425ae0 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncer.java @@ -29,8 +29,8 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.concurrent.LifecycleLock; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.http.client.HttpClient; @@ -71,9 +71,12 @@ public class ChangeRequestHttpSyncer private final String baseRequestPath; private final TypeReference> responseTypeReferences; private final long serverTimeoutMS; - private final long serverUnstabilityTimeout; private final long serverHttpTimeout; + private final Duration maxUnstableDuration; + private final Duration maxDelayBetweenSyncRequests; + private final Duration maxDurationToWaitForSync; + private final Listener listener; private final CountDownLatch initializationLatch = new CountDownLatch(1); @@ -86,10 +89,12 @@ public class ChangeRequestHttpSyncer private final LifecycleLock startStopLock = new LifecycleLock(); private final String logIdentity; - private long unstableStartTime = -1; private int consecutiveFailedAttemptCount = 0; - private long lastSuccessfulSyncTime = 0; - private long lastSyncTime = 0; + + private final Stopwatch sinceSyncerStart = Stopwatch.createUnstarted(); + private final Stopwatch sinceLastSyncRequest = Stopwatch.createUnstarted(); + private final Stopwatch sinceLastSyncSuccess = Stopwatch.createUnstarted(); + private final Stopwatch sinceUnstable = Stopwatch.createUnstarted(); @Nullable private ChangeRequestHistory.Counter counter = null; @@ -113,29 +118,30 @@ public class ChangeRequestHttpSyncer this.baseRequestPath = baseRequestPath; this.responseTypeReferences = responseTypeReferences; this.serverTimeoutMS = serverTimeoutMS; - this.serverUnstabilityTimeout = serverUnstabilityTimeout; this.serverHttpTimeout = serverTimeoutMS + HTTP_TIMEOUT_EXTRA_MS; this.listener = listener; this.logIdentity = StringUtils.format("%s_%d", baseServerURL, System.currentTimeMillis()); + + this.maxDurationToWaitForSync = Duration.millis(3 * serverHttpTimeout); + this.maxDelayBetweenSyncRequests = Duration.millis(3 * serverHttpTimeout + MAX_RETRY_BACKOFF); + this.maxUnstableDuration = Duration.millis(serverUnstabilityTimeout); } public void start() { - synchronized (startStopLock) { - if (!startStopLock.canStart()) { - throw new ISE("Can't start ChangeRequestHttpSyncer[%s].", logIdentity); + throw new ISE("Could not start sync for server[%s].", logIdentity); } try { - - log.info("Starting ChangeRequestHttpSyncer[%s].", logIdentity); + log.info("Starting sync for server[%s].", logIdentity); startStopLock.started(); } finally { startStopLock.exitStart(); } + sinceSyncerStart.restart(); addNextSyncToWorkQueue(); } } @@ -144,69 +150,91 @@ public class ChangeRequestHttpSyncer { synchronized (startStopLock) { if (!startStopLock.canStop()) { - throw new ISE("Can't stop ChangeRequestHttpSyncer[%s].", logIdentity); + throw new ISE("Could not stop sync for server[%s].", logIdentity); } try { - log.info("Stopping ChangeRequestHttpSyncer[%s].", logIdentity); + log.info("Stopping sync for server[%s].", logIdentity); } finally { startStopLock.exitStop(); } - log.info("Stopped ChangeRequestHttpSyncer[%s].", logIdentity); + log.info("Stopped sync for server[%s].", logIdentity); } } - /** Wait for first fetch of segment listing from server. */ - public boolean awaitInitialization(long timeout, TimeUnit timeUnit) throws InterruptedException - { - return initializationLatch.await(timeout, timeUnit); - } - /** - * This method returns the debugging information for printing, must not be used for any other purpose. + * Waits for the first successful sync with this server up to {@link #maxDurationToWaitForSync}. + */ + public boolean awaitInitialization() throws InterruptedException + { + return initializationLatch.await(maxDurationToWaitForSync.getMillis(), TimeUnit.MILLISECONDS); + } + + /** + * Waits upto 1 millisecond for the first successful sync with this server. + */ + public boolean isInitialized() throws InterruptedException + { + return initializationLatch.await(1, TimeUnit.MILLISECONDS); + } + + /** + * Returns debugging information for printing, must not be used for any other purpose. */ public Map getDebugInfo() { - long currTime = System.currentTimeMillis(); - - Object notSuccessfullySyncedFor; - if (lastSuccessfulSyncTime == 0) { - notSuccessfullySyncedFor = "Never Successfully Synced"; - } else { - notSuccessfullySyncedFor = (currTime - lastSuccessfulSyncTime) / 1000; - } return ImmutableMap.of( - "notSyncedForSecs", lastSyncTime == 0 ? "Never Synced" : (currTime - lastSyncTime) / 1000, - "notSuccessfullySyncedFor", notSuccessfullySyncedFor, + "millisSinceLastRequest", sinceLastSyncRequest.millisElapsed(), + "millisSinceLastSuccess", sinceLastSyncSuccess.millisElapsed(), "consecutiveFailedAttemptCount", consecutiveFailedAttemptCount, "syncScheduled", startStopLock.isStarted() ); } /** - * Exposed for monitoring use to see if sync is working fine and not stopped due to any coding bugs. If this - * ever returns false then caller of this method must create an alert and it should be looked into for any - * bugs. + * Whether this syncer should be reset. This method returning true typically + * indicates a problem with the sync scheduler. + * + * @return true if the delay since the last request to the server (or since + * syncer start in case of no request to the server) has exceeded + * {@link #maxDelayBetweenSyncRequests}. */ - public boolean isOK() + public boolean needsReset() { - return (System.currentTimeMillis() - lastSyncTime) < MAX_RETRY_BACKOFF + 3 * serverHttpTimeout; + if (sinceLastSyncRequest.isRunning()) { + return sinceLastSyncRequest.hasElapsed(maxDelayBetweenSyncRequests); + } else { + return sinceSyncerStart.hasElapsed(maxDelayBetweenSyncRequests); + } } - public long getServerHttpTimeout() + public long getUnstableTimeMillis() { - return serverHttpTimeout; + return consecutiveFailedAttemptCount <= 0 ? 0 : sinceUnstable.millisElapsed(); + } + + /** + * @return true if there have been no sync failures recently and the last + * successful sync was not more than {@link #maxDurationToWaitForSync} ago. + */ + public boolean isSyncedSuccessfully() + { + if (consecutiveFailedAttemptCount > 0) { + return false; + } else { + return sinceLastSyncSuccess.hasNotElapsed(maxDurationToWaitForSync); + } } private void sync() { if (!startStopLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { - log.info("Skipping sync() call for server[%s].", logIdentity); + log.info("Skipping sync for server[%s] as syncer has not started yet.", logIdentity); return; } - lastSyncTime = System.currentTimeMillis(); + sinceLastSyncRequest.restart(); try { final String req = getRequestString(); @@ -234,28 +262,27 @@ public class ChangeRequestHttpSyncer { synchronized (startStopLock) { if (!startStopLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { - log.info("Skipping sync() success for server[%s].", logIdentity); + log.info("Not handling response for server[%s] as syncer has not started yet.", logIdentity); return; } try { - if (responseHandler.getStatus() == HttpServletResponse.SC_NO_CONTENT) { + final int responseCode = responseHandler.getStatus(); + if (responseCode == HttpServletResponse.SC_NO_CONTENT) { log.debug("Received NO CONTENT from server[%s]", logIdentity); - lastSuccessfulSyncTime = System.currentTimeMillis(); + sinceLastSyncSuccess.restart(); return; - } else if (responseHandler.getStatus() != HttpServletResponse.SC_OK) { - handleFailure(new RE("Bad Sync Response.")); + } else if (responseCode != HttpServletResponse.SC_OK) { + handleFailure(new ISE("Received sync response [%d]", responseCode)); return; } - log.debug("Received sync response from [%s]", logIdentity); - + log.debug("Received sync response from server[%s]", logIdentity); ChangeRequestsSnapshot changes = smileMapper.readValue(stream, responseTypeReferences); - - log.debug("Finished reading sync response from [%s]", logIdentity); + log.debug("Finished reading sync response from server[%s]", logIdentity); if (changes.isResetCounter()) { - log.info("[%s] requested resetCounter for reason [%s].", logIdentity, changes.getResetCause()); + log.info("Server[%s] requested resetCounter for reason[%s].", logIdentity, changes.getResetCause()); counter = null; return; } @@ -270,29 +297,19 @@ public class ChangeRequestHttpSyncer if (initializationLatch.getCount() > 0) { initializationLatch.countDown(); - log.info("[%s] synced successfully for the first time.", logIdentity); + log.info("Server[%s] synced successfully for the first time.", logIdentity); } if (consecutiveFailedAttemptCount > 0) { consecutiveFailedAttemptCount = 0; - log.info("[%s] synced successfully.", logIdentity); + sinceUnstable.reset(); + log.info("Server[%s] synced successfully.", logIdentity); } - lastSuccessfulSyncTime = System.currentTimeMillis(); + sinceLastSyncSuccess.restart(); } catch (Exception ex) { - String logMsg = StringUtils.nonStrictFormat( - "Error processing sync response from [%s]. Reason [%s]", - logIdentity, - ex.getMessage() - ); - - if (incrementFailedAttemptAndCheckUnstabilityTimeout()) { - log.error(ex, logMsg); - } else { - log.info("Temporary Failure. %s", logMsg); - log.debug(ex, logMsg); - } + markServerUnstableAndAlert(ex, "Processing Response"); } finally { addNextSyncToWorkQueue(); @@ -305,7 +322,7 @@ public class ChangeRequestHttpSyncer { synchronized (startStopLock) { if (!startStopLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { - log.info("Skipping sync() failure for URL[%s].", logIdentity); + log.info("Not handling sync failure for server[%s] as syncer has not started yet.", logIdentity); return; } @@ -320,19 +337,12 @@ public class ChangeRequestHttpSyncer private void handleFailure(Throwable t) { - String logMsg = StringUtils.nonStrictFormat( - "failed to get sync response from [%s]. Return code [%s], Reason: [%s]", - logIdentity, + String logMsg = StringUtils.format( + "Handling response with code[%d], description[%s]", responseHandler.getStatus(), responseHandler.getDescription() ); - - if (incrementFailedAttemptAndCheckUnstabilityTimeout()) { - log.error(t, logMsg); - } else { - log.info("Temporary Failure. %s", logMsg); - log.debug(t, logMsg); - } + markServerUnstableAndAlert(t, logMsg); } }, executor @@ -340,16 +350,7 @@ public class ChangeRequestHttpSyncer } catch (Throwable th) { try { - String logMsg = StringUtils.nonStrictFormat( - "Fatal error while fetching segment list from [%s].", logIdentity - ); - - if (incrementFailedAttemptAndCheckUnstabilityTimeout()) { - log.makeAlert(th, logMsg).emit(); - } else { - log.info("Temporary Failure. %s", logMsg); - log.debug(th, logMsg); - } + markServerUnstableAndAlert(th, "Sending Request"); } finally { addNextSyncToWorkQueue(); @@ -384,47 +385,55 @@ public class ChangeRequestHttpSyncer try { if (consecutiveFailedAttemptCount > 0) { - long sleepMillis = Math.min( + long delayMillis = Math.min( MAX_RETRY_BACKOFF, RetryUtils.nextRetrySleepMillis(consecutiveFailedAttemptCount) ); - log.info("Scheduling next syncup in [%d] millis for server[%s].", sleepMillis, logIdentity); - executor.schedule(this::sync, sleepMillis, TimeUnit.MILLISECONDS); + log.info("Scheduling next sync for server[%s] in [%d] millis.", logIdentity, delayMillis); + executor.schedule(this::sync, delayMillis, TimeUnit.MILLISECONDS); } else { executor.execute(this::sync); } } catch (Throwable th) { if (executor.isShutdown()) { + log.warn(th, "Could not schedule sync for server[%s] because executor is stopped.", logIdentity); + } else { log.warn( th, - "Couldn't schedule next sync. [%s] is not being synced any more, probably because executor is stopped.", + "Could not schedule sync for server [%s]. This syncer will be reset automatically." + + " If the issue persists, try restarting this Druid service.", logIdentity ); - } else { - log.makeAlert( - th, - "Couldn't schedule next sync. [%s] is not being synced any more, restarting Druid process on that " - + "server might fix the issue.", - logIdentity - ).emit(); } } } } - private boolean incrementFailedAttemptAndCheckUnstabilityTimeout() + private void markServerUnstableAndAlert(Throwable throwable, String action) { - if (consecutiveFailedAttemptCount > 0 - && (System.currentTimeMillis() - unstableStartTime) > serverUnstabilityTimeout) { - return true; - } - if (consecutiveFailedAttemptCount++ == 0) { - unstableStartTime = System.currentTimeMillis(); + sinceUnstable.restart(); } - return false; + final long unstableSeconds = getUnstableTimeMillis() / 1000; + final String message = StringUtils.format( + "Sync failed for server[%s] while [%s]. Failed [%d] times in the last [%d] seconds.", + baseServerURL, action, consecutiveFailedAttemptCount, unstableSeconds + ); + + // Alert if unstable alert timeout has been exceeded + if (sinceUnstable.hasElapsed(maxUnstableDuration)) { + String alertMessage = StringUtils.format( + "%s. Try restarting the Druid process on server[%s].", + message, baseServerURL + ); + log.noStackTrace().makeAlert(throwable, alertMessage).emit(); + } else if (log.isDebugEnabled()) { + log.debug(throwable, message); + } else { + log.noStackTrace().info(throwable, message); + } } @VisibleForTesting diff --git a/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java index f69f0a84226..70ec2b2fa4a 100644 --- a/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java +++ b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java @@ -19,197 +19,439 @@ package org.apache.druid.client; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.discovery.NodeRole; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.RE; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.java.util.emitter.service.AlertEvent; +import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; +import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; import org.easymock.EasyMock; -import org.jboss.netty.buffer.ChannelBuffers; -import org.jboss.netty.handler.codec.http.DefaultHttpResponse; -import org.jboss.netty.handler.codec.http.HttpResponse; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.jboss.netty.handler.codec.http.HttpVersion; -import org.joda.time.Duration; +import org.joda.time.Period; +import org.junit.After; import org.junit.Assert; -import org.junit.BeforeClass; +import org.junit.Before; import org.junit.Test; -import java.io.ByteArrayInputStream; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; -/** - * - */ public class HttpServerInventoryViewTest { + private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); + private static final TypeReference> + TYPE_REF = HttpServerInventoryView.SEGMENT_LIST_RESP_TYPE_REF; - @BeforeClass - public static void setup() + private static final String EXEC_NAME_PREFIX = "InventoryViewTest"; + + private static final String METRIC_SUCCESS = "segment/serverview/sync/healthy"; + private static final String METRIC_UNSTABLE_TIME = "segment/serverview/sync/unstableTime"; + + private StubServiceEmitter serviceEmitter; + + private HttpServerInventoryView httpServerInventoryView; + private TestChangeRequestHttpClient> httpClient; + private TestExecutorFactory execHelper; + + private TestDruidNodeDiscovery druidNodeDiscovery; + private DruidNodeDiscoveryProvider druidNodeDiscoveryProvider; + + private Map> segmentsAddedToView; + private Map> segmentsRemovedFromView; + private Set removedServers; + + private AtomicBoolean inventoryInitialized; + + @Before + public void setup() { - EmittingLogger.registerEmitter(new NoopServiceEmitter()); - } + serviceEmitter = new StubServiceEmitter("test", "localhost"); + EmittingLogger.registerEmitter(serviceEmitter); - @Test(timeout = 60_000L) - public void testSimple() throws Exception - { - ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - - TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); - DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); + druidNodeDiscovery = new TestDruidNodeDiscovery(); + druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForService(DataNodeService.DISCOVERY_SERVICE_KEY)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); - final DataSegment segment1 = new DataSegment( - "test1", Intervals.of("2014/2015"), "v1", - null, null, null, null, 0, 0 + httpClient = new TestChangeRequestHttpClient<>(TYPE_REF, MAPPER); + execHelper = new TestExecutorFactory(); + inventoryInitialized = new AtomicBoolean(false); + + segmentsAddedToView = new HashMap<>(); + segmentsRemovedFromView = new HashMap<>(); + removedServers = new HashSet<>(); + + createInventoryView( + new HttpServerInventoryViewConfig(null, null, null) + ); + } + + @After + public void tearDown() + { + EasyMock.verify(druidNodeDiscoveryProvider); + if (httpServerInventoryView != null && httpServerInventoryView.isStarted()) { + httpServerInventoryView.stop(); + } + } + + @Test + public void testInitHappensAfterNodeViewInit() + { + httpServerInventoryView.start(); + Assert.assertTrue(httpServerInventoryView.isStarted()); + Assert.assertFalse(inventoryInitialized.get()); + + druidNodeDiscovery.markNodeViewInitialized(); + Assert.assertFalse(inventoryInitialized.get()); + + execHelper.finishInventoryInitialization(); + Assert.assertTrue(inventoryInitialized.get()); + + httpServerInventoryView.stop(); + } + + @Test + public void testStopShutsDownExecutors() + { + httpServerInventoryView.start(); + Assert.assertFalse(execHelper.syncExecutor.isShutdown()); + + httpServerInventoryView.stop(); + Assert.assertTrue(execHelper.syncExecutor.isShutdown()); + } + + @Test + public void testAddNodeStartsSync() + { + httpServerInventoryView.start(); + druidNodeDiscovery.markNodeViewInitialized(); + execHelper.finishInventoryInitialization(); + + final DiscoveryDruidNode druidNode = druidNodeDiscovery + .addNodeAndNotifyListeners("localhost"); + final DruidServer server = druidNode.toDruidServer(); + + Collection inventory = httpServerInventoryView.getInventory(); + Assert.assertEquals(1, inventory.size()); + Assert.assertTrue(inventory.contains(server)); + + execHelper.emitMetrics(); + serviceEmitter.verifyValue(METRIC_SUCCESS, 1); + serviceEmitter.verifyNotEmitted(METRIC_UNSTABLE_TIME); + + DataSegment segment = CreateDataSegments.ofDatasource("wiki").eachOfSizeInMb(500).get(0); + httpClient.completeNextRequestWith( + snapshotOf(new SegmentChangeRequestLoad(segment)) + ); + execHelper.sendSyncRequestAndHandleResponse(); + + DruidServer inventoryValue = httpServerInventoryView.getInventoryValue(server.getName()); + Assert.assertNotNull(inventoryValue); + Assert.assertEquals(1, inventoryValue.getTotalSegments()); + Assert.assertNotNull(inventoryValue.getSegment(segment.getId())); + + httpServerInventoryView.stop(); + } + + @Test + public void testRemoveNodeStopsSync() + { + httpServerInventoryView.start(); + druidNodeDiscovery.markNodeViewInitialized(); + execHelper.finishInventoryInitialization(); + + final DiscoveryDruidNode druidNode = druidNodeDiscovery + .addNodeAndNotifyListeners("localhost"); + final DruidServer server = druidNode.toDruidServer(); + + druidNodeDiscovery.removeNodesAndNotifyListeners(druidNode); + + Assert.assertNull(httpServerInventoryView.getInventoryValue(server.getName())); + + execHelper.emitMetrics(); + serviceEmitter.verifyNotEmitted(METRIC_SUCCESS); + serviceEmitter.verifyNotEmitted(METRIC_UNSTABLE_TIME); + + httpServerInventoryView.stop(); + } + + @Test(timeout = 60_000L) + public void testSyncSegmentLoadAndDrop() + { + httpServerInventoryView.start(); + druidNodeDiscovery.markNodeViewInitialized(); + execHelper.finishInventoryInitialization(); + + final DiscoveryDruidNode druidNode = druidNodeDiscovery + .addNodeAndNotifyListeners("localhost"); + final DruidServer server = druidNode.toDruidServer(); + + final DataSegment[] segments = + CreateDataSegments.ofDatasource("wiki") + .forIntervals(4, Granularities.DAY) + .eachOfSizeInMb(500) + .toArray(new DataSegment[0]); + + // Request 1: Load S1 + httpClient.completeNextRequestWith( + snapshotOf(new SegmentChangeRequestLoad(segments[0])) + ); + execHelper.sendSyncRequestAndHandleResponse(); + Assert.assertTrue(isAddedToView(server, segments[0])); + + // Request 2: Drop S1, Load S2, S3 + resetForNextSyncRequest(); + httpClient.completeNextRequestWith( + snapshotOf( + new SegmentChangeRequestDrop(segments[0]), + new SegmentChangeRequestLoad(segments[1]), + new SegmentChangeRequestLoad(segments[2]) + ) + ); + execHelper.sendSyncRequestAndHandleResponse(); + Assert.assertTrue(isRemovedFromView(server, segments[0])); + Assert.assertTrue(isAddedToView(server, segments[1])); + Assert.assertTrue(isAddedToView(server, segments[2])); + + // Request 3: reset the counter + resetForNextSyncRequest(); + httpClient.completeNextRequestWith( + new ChangeRequestsSnapshot<>( + true, + "Server requested reset", + ChangeRequestHistory.Counter.ZERO, + Collections.emptyList() + ) + ); + execHelper.sendSyncRequestAndHandleResponse(); + Assert.assertTrue(segmentsAddedToView.isEmpty()); + Assert.assertTrue(segmentsRemovedFromView.isEmpty()); + + // Request 4: Load S3, S4 + resetForNextSyncRequest(); + httpClient.completeNextRequestWith( + snapshotOf( + new SegmentChangeRequestLoad(segments[2]), + new SegmentChangeRequestLoad(segments[3]) + ) + ); + execHelper.sendSyncRequestAndHandleResponse(); + Assert.assertTrue(isRemovedFromView(server, segments[1])); + Assert.assertTrue(isAddedToView(server, segments[3])); + + DruidServer inventoryValue = httpServerInventoryView.getInventoryValue(server.getName()); + Assert.assertNotNull(inventoryValue); + Assert.assertEquals(2, inventoryValue.getTotalSegments()); + Assert.assertNotNull(inventoryValue.getSegment(segments[2].getId())); + Assert.assertNotNull(inventoryValue.getSegment(segments[3].getId())); + + // Verify node removal + druidNodeDiscovery.removeNodesAndNotifyListeners(druidNode); + + // test removal event with empty services + druidNodeDiscovery.removeNodesAndNotifyListeners( + new DiscoveryDruidNode( + new DruidNode("service", "host", false, 8080, null, true, false), + NodeRole.INDEXER, + Collections.emptyMap() + ) ); - final DataSegment segment2 = new DataSegment( - "test2", Intervals.of("2014/2015"), "v1", - null, null, null, null, 0, 0 - ); - - final DataSegment segment3 = new DataSegment( - "test3", Intervals.of("2014/2015"), "v1", - null, null, null, null, 0, 0 - ); - - final DataSegment segment4 = new DataSegment( - "test4", Intervals.of("2014/2015"), "v1", - null, null, null, null, 0, 0 - ); - - final DataSegment segment5 = new DataSegment( - "non-loading-datasource", Intervals.of("2014/2015"), "v1", - null, null, null, null, 0, 0 - ); - - TestHttpClient httpClient = new TestHttpClient( - ImmutableList.of( - Futures.immediateFuture( - new ByteArrayInputStream( - jsonMapper.writerWithType(HttpServerInventoryView.SEGMENT_LIST_RESP_TYPE_REF).writeValueAsBytes( - new ChangeRequestsSnapshot( - false, - null, - ChangeRequestHistory.Counter.ZERO, - ImmutableList.of( - new SegmentChangeRequestLoad(segment1) - ) - ) - ) - ) - ), - Futures.immediateFuture( - new ByteArrayInputStream( - jsonMapper.writerWithType(HttpServerInventoryView.SEGMENT_LIST_RESP_TYPE_REF).writeValueAsBytes( - new ChangeRequestsSnapshot( - false, - null, - ChangeRequestHistory.Counter.ZERO, - ImmutableList.of( - new SegmentChangeRequestDrop(segment1), - new SegmentChangeRequestLoad(segment2), - new SegmentChangeRequestLoad(segment3) - ) - ) - ) - ) - ), - Futures.immediateFuture( - new ByteArrayInputStream( - jsonMapper.writerWithType(HttpServerInventoryView.SEGMENT_LIST_RESP_TYPE_REF).writeValueAsBytes( - new ChangeRequestsSnapshot( - true, - "force reset counter", - ChangeRequestHistory.Counter.ZERO, - ImmutableList.of() - ) - ) - ) - ), - Futures.immediateFuture( - new ByteArrayInputStream( - jsonMapper.writerWithType(HttpServerInventoryView.SEGMENT_LIST_RESP_TYPE_REF).writeValueAsBytes( - new ChangeRequestsSnapshot( - false, - null, - ChangeRequestHistory.Counter.ZERO, - ImmutableList.of( - new SegmentChangeRequestLoad(segment3), - new SegmentChangeRequestLoad(segment4), - new SegmentChangeRequestLoad(segment5) - ) - ) - ) - ) + // test removal rogue node (announced a service as a DataNodeService but wasn't a DataNodeService at the key) + druidNodeDiscovery.removeNodesAndNotifyListeners( + new DiscoveryDruidNode( + new DruidNode("service", "host", false, 8080, null, true, false), + NodeRole.INDEXER, + ImmutableMap.of( + DataNodeService.DISCOVERY_SERVICE_KEY, + new LookupNodeService("lookyloo") ) ) ); - DiscoveryDruidNode druidNode = new DiscoveryDruidNode( - new DruidNode("service", "host", false, 8080, null, true, false), - NodeRole.HISTORICAL, - ImmutableMap.of( - DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0) - ) + Assert.assertTrue(removedServers.contains(server.getMetadata())); + Assert.assertNull(httpServerInventoryView.getInventoryValue(server.getName())); + + httpServerInventoryView.stop(); + } + + @Test + public void testSyncWhenRequestFailedToSend() + { + httpServerInventoryView.start(); + druidNodeDiscovery.markNodeViewInitialized(); + execHelper.finishInventoryInitialization(); + + druidNodeDiscovery.addNodeAndNotifyListeners("localhost"); + + httpClient.failToSendNextRequestWith(new ISE("Could not send request to server")); + execHelper.sendSyncRequest(); + + serviceEmitter.flush(); + execHelper.emitMetrics(); + serviceEmitter.verifyValue(METRIC_SUCCESS, 0); + + httpServerInventoryView.stop(); + } + + @Test + public void testSyncWhenErrorResponse() + { + httpServerInventoryView.start(); + druidNodeDiscovery.markNodeViewInitialized(); + execHelper.finishInventoryInitialization(); + + druidNodeDiscovery.addNodeAndNotifyListeners("localhost"); + + httpClient.completeNextRequestWith(InvalidInput.exception("failure on server")); + execHelper.sendSyncRequestAndHandleResponse(); + + serviceEmitter.flush(); + execHelper.emitMetrics(); + serviceEmitter.verifyValue(METRIC_SUCCESS, 0); + + httpServerInventoryView.stop(); + } + + @Test + public void testUnstableServerAlertsAfterTimeout() + { + // Create inventory with alert timeout as 0 ms + createInventoryView( + new HttpServerInventoryViewConfig(null, Period.millis(0), null) ); - HttpServerInventoryView httpServerInventoryView = new HttpServerInventoryView( - jsonMapper, + httpServerInventoryView.start(); + druidNodeDiscovery.markNodeViewInitialized(); + execHelper.finishInventoryInitialization(); + + druidNodeDiscovery.addNodeAndNotifyListeners("localhost"); + + serviceEmitter.flush(); + httpClient.completeNextRequestWith(InvalidInput.exception("failure on server")); + execHelper.sendSyncRequestAndHandleResponse(); + + List alerts = serviceEmitter.getAlerts(); + Assert.assertEquals(1, alerts.size()); + AlertEvent alert = alerts.get(0); + Assert.assertTrue(alert.getDescription().contains("Sync failed for server")); + + serviceEmitter.flush(); + execHelper.emitMetrics(); + serviceEmitter.verifyValue(METRIC_SUCCESS, 0); + + httpServerInventoryView.stop(); + } + + @Test(timeout = 60_000) + public void testInitWaitsForServerToSync() + { + httpServerInventoryView.start(); + druidNodeDiscovery.markNodeViewInitialized(); + druidNodeDiscovery.addNodeAndNotifyListeners("localhost"); + + ExecutorService initExecutor = Execs.singleThreaded(EXEC_NAME_PREFIX + "-init"); + + try { + initExecutor.submit(() -> execHelper.finishInventoryInitialization()); + + // Wait to ensure that init thread is in progress and waiting + Thread.sleep(1000); + Assert.assertFalse(inventoryInitialized.get()); + + // Finish sync of server + httpClient.completeNextRequestWith(snapshotOf()); + execHelper.sendSyncRequestAndHandleResponse(); + + // Wait for 10 seconds to ensure that init thread knows about server sync + Thread.sleep(10_000); + Assert.assertTrue(inventoryInitialized.get()); + } + catch (InterruptedException e) { + throw new ISE(e, "Interrupted"); + } + finally { + initExecutor.shutdownNow(); + } + } + + @Test(timeout = 60_000) + public void testInitDoesNotWaitForRemovedServerToSync() + { + httpServerInventoryView.start(); + druidNodeDiscovery.markNodeViewInitialized(); + DiscoveryDruidNode node = druidNodeDiscovery.addNodeAndNotifyListeners("localhost"); + + ExecutorService initExecutor = Execs.singleThreaded(EXEC_NAME_PREFIX + "-init"); + + try { + initExecutor.submit(() -> execHelper.finishInventoryInitialization()); + + // Wait to ensure that init thread is in progress and waiting + Thread.sleep(1000); + Assert.assertFalse(inventoryInitialized.get()); + + // Remove the node from discovery + druidNodeDiscovery.removeNodesAndNotifyListeners(node); + + // Wait for 10 seconds to ensure that init thread knows about server removal + Thread.sleep(10_000); + Assert.assertTrue(inventoryInitialized.get()); + } + catch (InterruptedException e) { + throw new ISE(e, "Interrupted"); + } + finally { + initExecutor.shutdownNow(); + } + } + + private void createInventoryView(HttpServerInventoryViewConfig config) + { + httpServerInventoryView = new HttpServerInventoryView( + MAPPER, httpClient, druidNodeDiscoveryProvider, - (pair) -> !pair.rhs.getDataSource().equals("non-loading-datasource"), - new HttpServerInventoryViewConfig(null, null, null), - "test" - ); - - CountDownLatch initializeCallback1 = new CountDownLatch(1); - - Map segmentAddLathces = ImmutableMap.of( - segment1.getId(), new CountDownLatch(1), - segment2.getId(), new CountDownLatch(1), - segment3.getId(), new CountDownLatch(1), - segment4.getId(), new CountDownLatch(1) - ); - - Map segmentDropLatches = ImmutableMap.of( - segment1.getId(), new CountDownLatch(1), - segment2.getId(), new CountDownLatch(1) + pair -> !pair.rhs.getDataSource().equals("non-loading-datasource"), + config, + serviceEmitter, + execHelper, + EXEC_NAME_PREFIX ); httpServerInventoryView.registerSegmentCallback( @@ -219,138 +461,60 @@ public class HttpServerInventoryViewTest @Override public ServerView.CallbackAction segmentAdded(DruidServerMetadata server, DataSegment segment) { - segmentAddLathces.get(segment.getId()).countDown(); + segmentsAddedToView.computeIfAbsent(server, s -> new HashSet<>()).add(segment); return ServerView.CallbackAction.CONTINUE; } @Override public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment) { - segmentDropLatches.get(segment.getId()).countDown(); + segmentsRemovedFromView.computeIfAbsent(server, s -> new HashSet<>()).add(segment); return ServerView.CallbackAction.CONTINUE; } @Override public ServerView.CallbackAction segmentViewInitialized() { - initializeCallback1.countDown(); + inventoryInitialized.set(true); return ServerView.CallbackAction.CONTINUE; } } ); - final CountDownLatch serverRemovedCalled = new CountDownLatch(1); httpServerInventoryView.registerServerRemovedCallback( Execs.directExecutor(), - new ServerView.ServerRemovedCallback() - { - @Override - public ServerView.CallbackAction serverRemoved(DruidServer server) - { - if (server.getName().equals("host:8080")) { - serverRemovedCalled.countDown(); - return ServerView.CallbackAction.CONTINUE; - } else { - throw new RE("Unknown server [%s]", server.getName()); - } - } + server -> { + removedServers.add(server.getMetadata()); + return ServerView.CallbackAction.CONTINUE; } ); - - httpServerInventoryView.start(); - - druidNodeDiscovery.listener.nodesAdded(ImmutableList.of(druidNode)); - - initializeCallback1.await(); - segmentAddLathces.get(segment1.getId()).await(); - segmentDropLatches.get(segment1.getId()).await(); - segmentAddLathces.get(segment2.getId()).await(); - segmentAddLathces.get(segment3.getId()).await(); - segmentAddLathces.get(segment4.getId()).await(); - segmentDropLatches.get(segment2.getId()).await(); - - DruidServer druidServer = httpServerInventoryView.getInventoryValue("host:8080"); - Assert.assertEquals( - ImmutableMap.of(segment3.getId(), segment3, segment4.getId(), segment4), - Maps.uniqueIndex(druidServer.iterateAllSegments(), DataSegment::getId) - ); - - druidNodeDiscovery.listener.nodesRemoved(ImmutableList.of(druidNode)); - - // test removal event with empty services - druidNodeDiscovery.listener.nodesRemoved( - ImmutableList.of( - new DiscoveryDruidNode( - new DruidNode("service", "host", false, 8080, null, true, false), - NodeRole.INDEXER, - Collections.emptyMap() - ) - ) - ); - - // test removal rogue node (announced a service as a DataNodeService but wasn't a DataNodeService at the key) - druidNodeDiscovery.listener.nodesRemoved( - ImmutableList.of( - new DiscoveryDruidNode( - new DruidNode("service", "host", false, 8080, null, true, false), - NodeRole.INDEXER, - ImmutableMap.of( - DataNodeService.DISCOVERY_SERVICE_KEY, - new LookupNodeService("lookyloo") - ) - ) - ) - ); - - serverRemovedCalled.await(); - Assert.assertNull(httpServerInventoryView.getInventoryValue("host:8080")); - - EasyMock.verify(druidNodeDiscoveryProvider); - - httpServerInventoryView.stop(); } - @Test(timeout = 60_000L) - public void testSyncMonitoring() + private boolean isAddedToView(DruidServer server, DataSegment segment) { - ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - - TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); - DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForService(DataNodeService.DISCOVERY_SERVICE_KEY)) - .andReturn(druidNodeDiscovery); - EasyMock.replay(druidNodeDiscoveryProvider); - - TestHttpClient httpClient = new TestHttpClient(ImmutableList.of()); - - HttpServerInventoryView httpServerInventoryView = new HttpServerInventoryView( - jsonMapper, - httpClient, - druidNodeDiscoveryProvider, - (pair) -> !pair.rhs.getDataSource().equals("non-loading-datasource"), - new HttpServerInventoryViewConfig(null, null, null), - "test" - ); - - httpServerInventoryView.start(); - httpServerInventoryView.serverAdded(makeServer("abc.com:8080")); - httpServerInventoryView.serverAdded(makeServer("xyz.com:8080")); - httpServerInventoryView.serverAdded(makeServer("lol.com:8080")); - Assert.assertEquals(3, httpServerInventoryView.getDebugInfo().size()); - httpServerInventoryView.syncMonitoring(); - Assert.assertEquals(3, httpServerInventoryView.getDebugInfo().size()); + return segmentsAddedToView.getOrDefault(server.getMetadata(), Collections.emptySet()) + .contains(segment); } - private DruidServer makeServer(String host) + private boolean isRemovedFromView(DruidServer server, DataSegment segment) { - return new DruidServer( - host, - host, - host, - 100_000_000L, - ServerType.HISTORICAL, - "__default_tier", - 50 + return segmentsRemovedFromView.getOrDefault(server.getMetadata(), Collections.emptySet()) + .contains(segment); + } + + private void resetForNextSyncRequest() + { + segmentsAddedToView.clear(); + segmentsRemovedFromView.clear(); + } + + private static ChangeRequestsSnapshot snapshotOf( + DataSegmentChangeRequest... requests + ) + { + return ChangeRequestsSnapshot.success( + ChangeRequestHistory.Counter.ZERO, + Arrays.asList(requests) ); } @@ -367,64 +531,97 @@ public class HttpServerInventoryViewTest @Override public void registerListener(Listener listener) { - listener.nodesAdded(ImmutableList.of()); - listener.nodeViewInitialized(); this.listener = listener; } + + /** + * Marks the node view as initialized and notifies the listeners. + */ + void markNodeViewInitialized() + { + listener.nodeViewInitialized(); + } + + /** + * Creates and adds a new node and notifies the listeners. + */ + DiscoveryDruidNode addNodeAndNotifyListeners(String host) + { + final DruidNode druidNode = new DruidNode("druid/historical", host, false, 8080, null, true, false); + DataNodeService dataNodeService = new DataNodeService("tier", 10L << 30, ServerType.HISTORICAL, 0); + final DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( + druidNode, + NodeRole.HISTORICAL, + ImmutableMap.of(DataNodeService.DISCOVERY_SERVICE_KEY, dataNodeService) + ); + listener.nodesAdded(ImmutableList.of(discoveryDruidNode)); + + return discoveryDruidNode; + } + + void removeNodesAndNotifyListeners(DiscoveryDruidNode... nodesToRemove) + { + listener.nodesRemoved(Arrays.asList(nodesToRemove)); + } } - private static class TestHttpClient implements HttpClient + /** + * Creates and retains a handle on the executors used by the inventory view. + *

    + * There are 4 types of tasks submitted to the two executors. Upon succesful + * completion, each of these tasks add another task to the execution queue. + *

    + * Tasks running on sync executor: + *

      + *
    1. send request to server (adds "handle response" to queue)
    2. + *
    3. handle response and execute callbacks (adds "send request" to queue)
    4. + *
    + *

    + * Tasks running on monitoring executor. + *

      + *
    1. check and reset unhealthy servers (adds self to queue)
    2. + *
    3. emit metrics (adds self to queue)
    4. + *
    + */ + private static class TestExecutorFactory implements ScheduledExecutorFactory { - BlockingQueue results; - AtomicInteger requestNum = new AtomicInteger(0); - - TestHttpClient(List resultsList) - { - results = new LinkedBlockingQueue<>(); - results.addAll(resultsList); - } + private BlockingExecutorService syncExecutor; + private BlockingExecutorService monitorExecutor; @Override - public ListenableFuture go( - Request request, - HttpResponseHandler httpResponseHandler - ) + public ScheduledExecutorService create(int corePoolSize, String nameFormat) { - throw new UnsupportedOperationException("Not Implemented."); + BlockingExecutorService executorService = new BlockingExecutorService(nameFormat); + final String syncExecutorPrefix = EXEC_NAME_PREFIX + "-%s"; + final String monitorExecutorPrefix = EXEC_NAME_PREFIX + "-monitor-%s"; + if (syncExecutorPrefix.equals(nameFormat)) { + syncExecutor = executorService; + } else if (monitorExecutorPrefix.equals(nameFormat)) { + monitorExecutor = executorService; + } + + return new WrappingScheduledExecutorService(nameFormat, executorService, false); } - @Override - public ListenableFuture go( - Request request, - HttpResponseHandler httpResponseHandler, - Duration duration - ) + void sendSyncRequestAndHandleResponse() { - if (requestNum.getAndIncrement() == 0) { - //fail first request immediately - throw new RuntimeException("simulating couldn't send request to server for some reason."); - } + syncExecutor.finishNextPendingTasks(2); + } - if (requestNum.get() == 2) { - //fail scenario where request is sent to server but we got an unexpected response. - HttpResponse httpResponse = new DefaultHttpResponse( - HttpVersion.HTTP_1_1, - HttpResponseStatus.INTERNAL_SERVER_ERROR - ); - httpResponse.setContent(ChannelBuffers.buffer(0)); - httpResponseHandler.handleResponse(httpResponse, null); - return Futures.immediateFailedFuture(new RuntimeException("server error")); - } + void sendSyncRequest() + { + syncExecutor.finishNextPendingTask(); + } - HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - httpResponse.setContent(ChannelBuffers.buffer(0)); - httpResponseHandler.handleResponse(httpResponse, null); - try { - return results.take(); - } - catch (InterruptedException ex) { - throw new RE(ex, "Interrupted."); - } + void finishInventoryInitialization() + { + syncExecutor.finishNextPendingTask(); + } + + void emitMetrics() + { + // Finish 1 task for check and reset, 1 for metric emission + monitorExecutor.finishNextPendingTasks(2); } } } diff --git a/server/src/test/java/org/apache/druid/client/TestChangeRequestHttpClient.java b/server/src/test/java/org/apache/druid/client/TestChangeRequestHttpClient.java new file mode 100644 index 00000000000..344497ee06d --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/TestChangeRequestHttpClient.java @@ -0,0 +1,164 @@ +/* + * 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.druid.client; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.ErrorResponse; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.jboss.netty.buffer.ChannelBuffers; +import org.jboss.netty.handler.codec.http.DefaultHttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.jboss.netty.handler.codec.http.HttpVersion; +import org.joda.time.Duration; + +import java.io.ByteArrayInputStream; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +/** + * Test implementation of {@link HttpClient} that can be used to test + * {@link org.apache.druid.server.coordination.ChangeRequestHttpSyncer}. + */ +public class TestChangeRequestHttpClient implements HttpClient +{ + private final ObjectMapper mapper; + private final TypeReference typeReference; + private final BlockingQueue> results = new LinkedBlockingQueue<>(); + + private final AtomicInteger requestCount = new AtomicInteger(0); + + public TestChangeRequestHttpClient(TypeReference typeReference, ObjectMapper mapper) + { + this.mapper = mapper; + this.typeReference = typeReference; + } + + public void failToSendNextRequestWith(RuntimeException error) + { + results.add(new ResultHolder<>(null, error, null)); + } + + public void completeNextRequestWith(DruidException druidException) + { + results.add(new ResultHolder<>(null, null, druidException)); + } + + public void completeNextRequestWith(R result) + { + results.add(new ResultHolder<>(() -> result, null, null)); + } + + public boolean hasPendingResults() + { + return !results.isEmpty(); + } + + @Override + public ListenableFuture go( + Request request, + HttpResponseHandler httpResponseHandler + ) + { + throw new UnsupportedOperationException("Not Implemented."); + } + + @Override + public ListenableFuture go( + Request request, + HttpResponseHandler httpResponseHandler, + Duration duration + ) + { + final int currentRequest = requestCount.getAndIncrement(); + + final ResultHolder nextResult = results.poll(); + if (nextResult == null) { + throw new ISE("No known response for request [%d]", currentRequest); + } else if (nextResult.clientError != null) { + throw nextResult.clientError; + } else if (nextResult.serverError != null) { + HttpResponse errorResponse = buildErrorResponse(nextResult.serverError); + httpResponseHandler.handleResponse(errorResponse, null); + return (ListenableFuture) Futures.immediateFuture(new ByteArrayInputStream(new byte[0])); + } else { + HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + httpResponse.setContent(ChannelBuffers.buffer(0)); + httpResponseHandler.handleResponse(httpResponse, null); + } + + try { + ByteArrayInputStream resultBytes = new ByteArrayInputStream( + mapper.writerFor(typeReference).writeValueAsBytes(nextResult.supplier.get()) + ); + return (ListenableFuture) Futures.immediateFuture(resultBytes); + } + catch (Exception e) { + throw new RE(e, "Error while sending HTTP response: %s", e.getMessage()); + } + } + + private HttpResponse buildErrorResponse(DruidException druidException) + { + HttpResponse httpResponse = new DefaultHttpResponse( + HttpVersion.HTTP_1_1, + HttpResponseStatus.valueOf(druidException.getStatusCode()) + ); + httpResponse.setContent(ChannelBuffers.buffer(0)); + + ErrorResponse errorResponse = druidException.toErrorResponse(); + try { + httpResponse.setContent(ChannelBuffers.copiedBuffer(mapper.writeValueAsBytes(errorResponse))); + return httpResponse; + } + catch (JsonProcessingException e) { + throw new ISE("Error while serializing given response"); + } + } + + private static class ResultHolder + { + final Supplier supplier; + final RuntimeException clientError; + final DruidException serverError; + + ResultHolder( + Supplier supplier, + RuntimeException clientError, + DruidException serverError + ) + { + this.supplier = supplier; + this.clientError = clientError; + this.serverError = serverError; + } + } +} From 277b35725619d6d6b63b5c925a72ffb19a37d703 Mon Sep 17 00:00:00 2001 From: imply-cheddar <86940447+imply-cheddar@users.noreply.github.com> Date: Thu, 6 Jul 2023 18:14:23 +0900 Subject: [PATCH 70/74] Optimize IntervalIterator (#14530) UniformGranularityTest's test to test a large number of intervals runs through 10 years of 1 second intervals. This pushes a lot of stuff through IntervalIterator and shows up in terms of test runtime as one of the hottest tests. Most of the time is going to constructing jodatime objects because it is doing things with DateTime objects instead of millis. Change the calls to use millis instead and things go faster. --- .../DetermineHashedPartitionsJobTest.java | 29 ++++++++++++------- .../util/common/granularity/Granularity.java | 15 ++++++---- .../granularity/IntervalsByGranularity.java | 17 ++++++----- .../common/granularity/PeriodGranularity.java | 1 + .../granularity/UniformGranularityTest.java | 3 +- 5 files changed, 40 insertions(+), 25 deletions(-) diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java index d80a60306ca..5f7b0157fba 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -47,6 +47,7 @@ import org.junit.runners.Parameterized; import javax.annotation.Nullable; import java.io.File; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -57,20 +58,17 @@ public class DetermineHashedPartitionsJobTest { private HadoopDruidIndexerConfig indexerConfig; private int expectedNumTimeBuckets; - private int[] expectedNumOfShards; + private ArrayList expectedNumOfShards; private int errorMargin; @Parameterized.Parameters(name = "File={0}, TargetPartitionSize={1}, Interval={2}, ErrorMargin={3}, NumTimeBuckets={4}, NumShards={5}, SegmentGranularity={6}") public static Collection data() { - int[] first = new int[1]; - Arrays.fill(first, 13); - int[] second = new int[6]; - Arrays.fill(second, 1); - int[] third = new int[6]; - Arrays.fill(third, 13); - third[2] = 12; - third[5] = 11; + ArrayList first = makeListOf(1, 13); + ArrayList second = makeListOf(6, 1); + ArrayList third = makeListOf(6, 13); + third.set(2, 12); + third.set(5, 11); return Arrays.asList( new Object[][]{ @@ -144,7 +142,7 @@ public class DetermineHashedPartitionsJobTest String interval, int errorMargin, int expectedNumTimeBuckets, - int[] expectedNumOfShards, + ArrayList expectedNumOfShards, Granularity segmentGranularity, @Nullable HashPartitionFunction partitionFunction ) @@ -254,7 +252,7 @@ public class DetermineHashedPartitionsJobTest int i = 0; for (Map.Entry> entry : shardSpecs.entrySet()) { Assert.assertEquals( - expectedNumOfShards[i++], + expectedNumOfShards.get(i++), entry.getValue().size(), errorMargin ); @@ -264,4 +262,13 @@ public class DetermineHashedPartitionsJobTest } } } + + private static ArrayList makeListOf(int capacity, int value) + { + ArrayList retVal = new ArrayList<>(); + for (int i = 0; i < capacity; ++i) { + retVal.add(value); + } + return retVal; + } } diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java index 6f5d9fb61e1..ca307886a79 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java @@ -150,6 +150,11 @@ public abstract class Granularity implements Cacheable */ public abstract boolean isAligned(Interval interval); + public DateTimeZone getTimeZone() + { + return DateTimeZone.UTC; + } + public DateTime bucketEnd(DateTime time) { return increment(bucketStart(time)); @@ -255,21 +260,21 @@ public abstract class Granularity implements Cacheable { private final Interval inputInterval; - private DateTime currStart; - private DateTime currEnd; + private long currStart; + private long currEnd; private IntervalIterator(Interval inputInterval) { this.inputInterval = inputInterval; - currStart = bucketStart(inputInterval.getStart()); + currStart = bucketStart(inputInterval.getStartMillis()); currEnd = increment(currStart); } @Override public boolean hasNext() { - return currStart.isBefore(inputInterval.getEnd()); + return currStart < inputInterval.getEndMillis(); } @Override @@ -278,7 +283,7 @@ public abstract class Granularity implements Cacheable if (!hasNext()) { throw new NoSuchElementException("There are no more intervals"); } - Interval retVal = new Interval(currStart, currEnd); + Interval retVal = new Interval(currStart, currEnd, getTimeZone()); currStart = currEnd; currEnd = increment(currStart); diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/IntervalsByGranularity.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/IntervalsByGranularity.java index ea742a4ec8d..1e5f0240fd7 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/IntervalsByGranularity.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/IntervalsByGranularity.java @@ -68,14 +68,15 @@ public class IntervalsByGranularity // intervals will be returned, both with the same value 2013-01-01T00:00:00.000Z/2013-02-01T00:00:00.000Z. // Thus dups can be created given the right conditions.... final SettableSupplier previous = new SettableSupplier<>(); - return FluentIterable.from(sortedNonOverlappingIntervals).transformAndConcat(granularity::getIterable) - .filter(interval -> { - if (previous.get() != null && previous.get().equals(interval)) { - return false; - } - previous.set(interval); - return true; - }).iterator(); + return FluentIterable.from(sortedNonOverlappingIntervals) + .transformAndConcat(granularity::getIterable) + .filter(interval -> { + if (previous.get() != null && previous.get().equals(interval)) { + return false; + } + previous.set(interval); + return true; + }).iterator(); } } } diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java index 09960e33cda..4f9acefd284 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java @@ -79,6 +79,7 @@ public class PeriodGranularity extends Granularity implements JsonSerializable return period; } + @Override @JsonProperty("timeZone") public DateTimeZone getTimeZone() { diff --git a/server/src/test/java/org/apache/druid/segment/indexing/granularity/UniformGranularityTest.java b/server/src/test/java/org/apache/druid/segment/indexing/granularity/UniformGranularityTest.java index 0c19724883e..5ec95774729 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/granularity/UniformGranularityTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/granularity/UniformGranularityTest.java @@ -27,6 +27,7 @@ import com.google.common.collect.Lists; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.DurationGranularity; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.joda.time.Interval; @@ -343,7 +344,7 @@ public class UniformGranularityTest { // just make sure that intervals for uniform spec are not materialized (causing OOM) when created final GranularitySpec spec = new UniformGranularitySpec( - Granularities.SECOND, + new DurationGranularity(1000, 0), null, Collections.singletonList( Intervals.of("2012-01-01T00Z/P10Y") From 5fc122a144cbb9dcc3471e518872ef2d2a135504 Mon Sep 17 00:00:00 2001 From: imply-cheddar <86940447+imply-cheddar@users.noreply.github.com> Date: Fri, 7 Jul 2023 01:20:32 +0900 Subject: [PATCH 71/74] Add window-focused tests from Drill (#13773) This commit borrows some test definitions from Drill's test suite and tries to use them to flesh out the full validation of window function capbilities. In order to be able to run these tests, we also add the ability to run a Scan operation against segments, which also meant an implementation of RowsAndColumns for frames. --- LICENSE | 3 + extensions-core/parquet-extensions/pom.xml | 2 +- .../data/input/parquet/ParquetToJson.java | 69 + .../parquet/simple/ParquetGroupConverter.java | 24 +- .../data/input/parquet/ParquetToJsonTest.java | 79 + .../src/test/resources/smlTbl.parquet | Bin 0 -> 3271 bytes licenses.yaml | 11 + .../apache/druid/error/DruidException.java | 24 +- .../columnar/ComplexFrameColumnReader.java | 68 +- .../columnar/DoubleFrameColumnReader.java | 72 +- .../read/columnar/FloatFrameColumnReader.java | 72 +- .../read/columnar/FrameColumnReader.java | 6 + .../read/columnar/LongFrameColumnReader.java | 73 +- .../columnar/StringFrameColumnReader.java | 51 +- .../DruidDefaultSerializersModule.java | 18 + .../operator/LimitTimeIntervalOperator.java | 78 + .../operator/NaivePartitioningOperator.java | 99 +- .../query/operator/OperatorSequence.java | 1 + .../druid/query/operator/ScanOperator.java | 118 + .../query/operator/ScanOperatorFactory.java | 136 + .../SegmentToRowsAndColumnsOperator.java | 7 +- .../query/operator/SequenceOperator.java | 144 +- .../query/operator/WindowOperatorQuery.java | 110 +- ...WindowOperatorQueryQueryRunnerFactory.java | 38 +- .../join/SortedInnerJoinOperator.java | 63 +- .../rowsandcols/ArrayListRowsAndColumns.java | 2 +- .../LazilyDecoratedRowsAndColumns.java | 361 + .../MapOfColumnsRowsAndColumns.java | 35 + .../rowsandcols/column/ObjectArrayColumn.java | 1 + .../accessor/DoubleColumnAccessorBase.java | 74 + .../accessor/FloatColumnAccessorBase.java | 74 + .../accessor/LongColumnAccessorBase.java | 74 + .../ObjectColumnAccessorBase.java | 4 +- .../concrete/ColumnHolderRACColumn.java | 188 + .../concrete/FrameRowsAndColumns.java | 86 + .../QueryableIndexRowsAndColumns.java | 106 + .../DefaultColumnSelectorFactoryMaker.java | 352 +- .../DefaultRowsAndColumnsDecorator.java | 149 + .../semantic/RowsAndColumnsDecorator.java | 71 + .../semantic/WireTransferable.java | 37 + .../druid/segment/DimensionSelector.java | 7 +- .../druid/segment/QueryableIndexSegment.java | 18 +- .../query/operator/OperatorSequenceTest.java | 8 +- .../query/operator/OperatorTestHelper.java | 64 +- .../operator/ScanOperatorFactoryTest.java | 296 + .../SegmentToRowsAndColumnsOperatorTest.java | 2 +- .../query/operator/SequenceOperatorTest.java | 7 +- .../operator/WindowOperatorQueryTest.java | 18 +- .../operator/WindowProcessorOperatorTest.java | 2 +- .../query/rowsandcols/TestRowsAndColumns.java | 73 + .../semantic/RowsAndColumnsDecoratorTest.java | 243 + .../semantic/TestRowsAndColumnsDecorator.java | 144 + .../druid/sql/calcite/rel/DruidQuery.java | 19 +- .../druid/sql/calcite/rel/Windowing.java | 77 +- .../sql/calcite/CalciteWindowQueryTest.java | 2 +- .../sql/calcite/DrillWindowQueryTest.java | 243 + .../tests/window/wikipediaScanWindow.sqlTest | 255 +- .../drill/window/datasources/allData.csv | 11196 ++++++++++++++++ .../window/datasources/allTypsUniq.parquet | Bin 0 -> 2364 bytes .../datasources/allTypsUniq.parquet.json | 22 + .../window/datasources/fewRowsAllData.parquet | Bin 0 -> 3990 bytes .../datasources/fewRowsAllData.parquet.json | 78 + .../window/datasources/forViewCrn.parquet | Bin 0 -> 2383 bytes .../datasources/forViewCrn.parquet.json | 30 + .../drill/window/datasources/smlTbl.parquet | Bin 0 -> 3271 bytes .../window/datasources/smlTbl.parquet.json | 56 + .../drill/window/datasources/t_alltype.csv | 145 + .../window/datasources/t_alltype.parquet | Bin 0 -> 12340 bytes .../window/datasources/t_alltype.parquet.json | 145 + .../window/datasources/tblWnulls.parquet | Bin 0 -> 395 bytes .../window/datasources/tblWnulls.parquet.json | 30 + .../window/queries/aggregates/aggOWnFn_1.e | 22 + .../window/queries/aggregates/aggOWnFn_1.q | 12 + .../window/queries/aggregates/aggOWnFn_2.e | 22 + .../window/queries/aggregates/aggOWnFn_2.q | 1 + 75 files changed, 15852 insertions(+), 365 deletions(-) create mode 100644 extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetToJson.java create mode 100644 extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/ParquetToJsonTest.java create mode 100644 extensions-core/parquet-extensions/src/test/resources/smlTbl.parquet create mode 100644 processing/src/main/java/org/apache/druid/query/operator/LimitTimeIntervalOperator.java create mode 100644 processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java create mode 100644 processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/DoubleColumnAccessorBase.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/FloatColumnAccessorBase.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/LongColumnAccessorBase.java rename processing/src/main/java/org/apache/druid/query/rowsandcols/column/{ => accessor}/ObjectColumnAccessorBase.java (95%) create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnHolderRACColumn.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/WireTransferable.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/TestRowsAndColumns.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java create mode 100644 sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java create mode 100755 sql/src/test/resources/drill/window/datasources/allData.csv create mode 100644 sql/src/test/resources/drill/window/datasources/allTypsUniq.parquet create mode 100644 sql/src/test/resources/drill/window/datasources/allTypsUniq.parquet.json create mode 100644 sql/src/test/resources/drill/window/datasources/fewRowsAllData.parquet create mode 100644 sql/src/test/resources/drill/window/datasources/fewRowsAllData.parquet.json create mode 100644 sql/src/test/resources/drill/window/datasources/forViewCrn.parquet create mode 100644 sql/src/test/resources/drill/window/datasources/forViewCrn.parquet.json create mode 100644 sql/src/test/resources/drill/window/datasources/smlTbl.parquet create mode 100644 sql/src/test/resources/drill/window/datasources/smlTbl.parquet.json create mode 100644 sql/src/test/resources/drill/window/datasources/t_alltype.csv create mode 100644 sql/src/test/resources/drill/window/datasources/t_alltype.parquet create mode 100644 sql/src/test/resources/drill/window/datasources/t_alltype.parquet.json create mode 100644 sql/src/test/resources/drill/window/datasources/tblWnulls.parquet create mode 100644 sql/src/test/resources/drill/window/datasources/tblWnulls.parquet.json create mode 100644 sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_1.e create mode 100644 sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_1.q create mode 100644 sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_2.e create mode 100644 sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_2.q diff --git a/LICENSE b/LICENSE index 4b63a77502e..b14bc8a2dda 100644 --- a/LICENSE +++ b/LICENSE @@ -282,6 +282,9 @@ SOURCE/JAVA-CORE This product contains SystemInfo methods adapted from oshi * processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java + This product contains test cases adapted from Test Framework for Apache Drill (https://github.com/apache/drill-test-framework). + * sql/src/test/resources/drill/window + MIT License ================================ diff --git a/extensions-core/parquet-extensions/pom.xml b/extensions-core/parquet-extensions/pom.xml index 8f0e43c887a..88978943a6e 100644 --- a/extensions-core/parquet-extensions/pom.xml +++ b/extensions-core/parquet-extensions/pom.xml @@ -447,7 +447,7 @@ - 1.13.0 + 1.13.0 diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetToJson.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetToJson.java new file mode 100644 index 00000000000..5cc40df45c2 --- /dev/null +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetToJson.java @@ -0,0 +1,69 @@ +/* + * 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.druid.data.input.parquet; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SequenceWriter; +import org.apache.druid.data.input.parquet.simple.ParquetGroupConverter; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.IAE; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.hadoop.example.GroupReadSupport; + +import java.io.File; + +/** + * Converts parquet files into new-deliminated JSON object files. Takes a single argument (an input directory) + * and processes all files that end with a ".parquet" extension. Writes out a new file in the same directory named + * by appending ".json" to the old file name. Will overwrite any output file that already exists. + */ +public class ParquetToJson +{ + + public static void main(String[] args) throws Exception + { + if (args.length != 1) { + throw new IAE("Usage: directory"); + } + + ParquetGroupConverter converter = new ParquetGroupConverter(true); + ObjectMapper mapper = new DefaultObjectMapper(); + + File[] inputFiles = new File(args[0]).listFiles( + pathname -> pathname.getName().endsWith(".parquet") + ); + for (File inputFile : inputFiles) { + File outputFile = new File(inputFile.getAbsolutePath() + ".json"); + + try ( + final org.apache.parquet.hadoop.ParquetReader reader = org.apache.parquet.hadoop.ParquetReader + .builder(new GroupReadSupport(), new Path(inputFile.toURI())) + .build(); + final SequenceWriter writer = mapper.writer().withRootValueSeparator("\n").writeValues(outputFile) + ) { + Group group; + while ((group = reader.read()) != null) { + writer.write(converter.convertGroup(group)); + } + } + } + } +} diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java index 13e21fa2a0d..d2bd643304a 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java @@ -40,11 +40,12 @@ import java.nio.ByteOrder; import java.nio.IntBuffer; import java.util.ArrayList; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -class ParquetGroupConverter +public class ParquetGroupConverter { private static final int JULIAN_EPOCH_OFFSET_DAYS = 2_440_588; private static final long MILLIS_IN_DAY = TimeUnit.DAYS.toMillis(1); @@ -359,7 +360,6 @@ class ParquetGroupConverter return g.getInteger(fieldIndex, index); case INT_64: return g.getLong(fieldIndex, index); - // todo: idk wtd about unsigned case UINT_8: case UINT_16: return g.getInteger(fieldIndex, index); @@ -493,11 +493,29 @@ class ParquetGroupConverter private final boolean binaryAsString; - ParquetGroupConverter(boolean binaryAsString) + public ParquetGroupConverter(boolean binaryAsString) { this.binaryAsString = binaryAsString; } + /** + * Recursively converts a group into native Java Map + * + * @param g the group + * @return the native Java object + */ + public Object convertGroup(Group g) + { + Map retVal = new LinkedHashMap<>(); + + for (Type field : g.getType().getFields()) { + final String fieldName = field.getName(); + retVal.put(fieldName, convertField(g, fieldName)); + } + + return retVal; + } + /** * Convert a parquet group field as though it were a map. Logical types of 'list' and 'map' will be transformed * into java lists and maps respectively ({@link ParquetGroupConverter#convertLogicalList} and diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/ParquetToJsonTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/ParquetToJsonTest.java new file mode 100644 index 00000000000..dd191e36dd1 --- /dev/null +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/ParquetToJsonTest.java @@ -0,0 +1,79 @@ +/* + * 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.druid.data.input.parquet; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.IAE; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.InputStream; +import java.nio.file.Files; +import java.util.List; + +@SuppressWarnings("ALL") +public class ParquetToJsonTest +{ + @ClassRule + public static TemporaryFolder tmp = new TemporaryFolder(); + + @Test + public void testSanity() throws Exception + { + final File tmpDir = tmp.newFolder(); + try (InputStream in = new BufferedInputStream(ClassLoader.getSystemResourceAsStream("smlTbl.parquet"))) { + Files.copy(in, tmpDir.toPath().resolve("smlTbl.parquet")); + } + + ParquetToJson.main(new String[]{tmpDir.toString()}); + + DefaultObjectMapper mapper = DefaultObjectMapper.INSTANCE; + List objs = mapper.readerFor(Object.class).readValues(new File(tmpDir, "smlTbl.parquet.json")).readAll(); + + Assert.assertEquals(56, objs.size()); + Assert.assertEquals( + ImmutableMap + .builder() + .put("col_int", 8122) + .put("col_bgint", 817200) + .put("col_char_2", "IN") + .put("col_vchar_52", "AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB") + .put("col_tmstmp", 1409617682418L) + .put("col_dt", 422717616000000L) + .put("col_booln", false) + .put("col_dbl", 12900.48) + .put("col_tm", 33109170) + .build(), + objs.get(0) + ); + } + + @Test + public void testInputValidation() + { + Assert.assertThrows(IAE.class, () -> ParquetToJson.main(new String[]{})); + Assert.assertThrows(IAE.class, () -> ParquetToJson.main(new String[]{"a", "b"})); + } +} diff --git a/extensions-core/parquet-extensions/src/test/resources/smlTbl.parquet b/extensions-core/parquet-extensions/src/test/resources/smlTbl.parquet new file mode 100644 index 0000000000000000000000000000000000000000..831e1d5c688d3a18d9327908b9e364c8d108cdba GIT binary patch literal 3271 zcmcgveN+_J6~Axx&F&kzs|#^915!l`djL@}Vx(c!SrlDHSP+P!$08~!T96eHsb4W_ z{E8oiXl$ZVqvU{*M2|6k#ULU_(HQGV6-jImzo?C;32jBd2-7>eplFZ(cFw%nci+9g z`+oN~Z{Ex%+LMNeAYRgmS9M{c3%h!e`Af8Em>wir@aTzZ0X!N>(>)G@(JJiN401J<`t8ZUX)#690x?#Dh5}fh<8vqG7Ha-?$Nu&;P0tg&v zB(T}6suRKhHoxDYUR*>#qxz3{9o+ZnPmH|YV!fh)Dx&;JHVeLSb%XzVt>!va{X|lxGr?{1^<$8IPC3+g` z#LL7Kyg0`YJtXsbi0365Zzn!NARQY=jDrG!j!r;RPbb9^Bf4#Z6CAt`jN_vPr(mR$ zCi9_uA0eG4B=J6cC?p9IKSeMiz&?c!IHoWCp<#v8sQv$Gle(>eS~weMmEmwxoQh&h*;hIX^L>9zU+v7nM08x z>@V}Bj((XZ+OsgFFpqo@$jhFZhHHFunlC^!iG(p}eSoIMr1=4iMRT0nbZBNQiJYPN zV$&i46cSlq2K5r@Xn=|acPA1$x}77j-%2W4N&OzK9b#W;wl^`yfE6^JU+&SISL3hj(cBuK#H#ZG-4SC2q<*p{-+r?jNI$n4>R zTi3^9sS<0Bf@39EdgoJ`=@H}KH>-v^(Epzsch(KT--mau^^GCS=!UVlX=SZZINF76 z{!82BvBW$D7w{vKw2DEmKgVCZf-5%|T1=5>`)|#%$p~mQg|EP%Hd8P>9?5|1D0oO} z;1`pG20SrIxSf9a>uuvOvfU&>`@l$Jd#U|$KUtJa0nkx_iQigR+lYxH6V>F}6*rN$ z`t#{{ZQD{8@AzamVUcbsXzp`xqi`5k81C0q-HWGrCluVTZ=@-nq8mx2r|9ja3NUO% zA1Dx#r$GM$3LMA2-|rPL<||ORT!Bx~PFt<;^gh2+f$s~@nx(*`Yz2xBD)8z?1)i){ zpcxUp-%?=FF$Jz3SD-CJfqSbISoyI6YgZz=Df;YM1sXn8V9pFogP}#c6qvPD0S8Vr zZL*y-Q+S%ld^gyCdZ`0ZUv5QQeX+vPM9swu+zwF z_MbcrHSiPrblW}%k%*^uQ@E#eeZT!b_>QjEo=1aNq3;&UdVMzT*4j;Amsp8wgxB?` zIdbPxX`1T6dR^b=8u8i%d5_=VA8LMeM?TveJoVt3^K$BMkJJph(;}DqPsxZ9m&r-T z`@Az?-(h*`)~e>#F=ymkOEC5FB6$eI2I-+3_Mc$%SFWVT1a7;E1l z&pVa(-qb5CGRdj=@nE(5Zr0y_37NQG4ngl1%I4v(gDQ1=*l_R`gw|(_+zWnt^ zF-L!M^Nj4ks(Ki!WUMhb&c8+`iAJ&XTbZ!L=6l;@#mEg<7r0Eut7X*S7A>ixu~05M zH-7ko*)4LH)ZQ#WEB*E>Rq|5eO}&eP%xq)NZ}^7bnzpm@45Svp{wM^{S9Xhx%!#v1 zUn(P=eXf!3J^l5p`{%z?9`1Mj*tPDvzEciIw2@jDuFA9}G_P$0J$hoiE^}iv zH)!2tc>O{K7{}yuO=nZc)Q8@j*39q5Q5gOQ8Ys@B%eK3q$=Rl>{Z}*G)m7_Q%O1kIW;$88(Y z_R{Q`nbUKpy%vO9h&E)@hVtEG!`y=!&K;MRy)ZA^71Wtnuf=8L1sQSYd73#VCu=^4 zUTC}L&6ttpVc`zW%MR)R7MHKT?=CCzN23yFv6gi8_Qba8uQY*9l&$p&^FZCB` zC*)ZfZqRLm^9?pCxL{v&TWs>2AI)ctG2T4H(zc}^4$(V>=$ zJjU3h|Mz>``jgKyCA!_1E-4;kEj}p4OWu6e`<7?gciu!#MbxsMq8GJ);k{vP^79wr zIqkI@?M5Y9QqXOap6#^O>fZjMSNEHX%b<(gHUqkmT-OHh!d`2w>;D3|yzDM=xGw7Y jBe@qIZcEwkEG`K4a!t=&uxM7^!0cSm>i`n*FU)@dL%Pwj literal 0 HcmV?d00001 diff --git a/licenses.yaml b/licenses.yaml index 8efba06fe84..48924ec710c 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -174,6 +174,17 @@ source_paths: --- +name: Test Framework for Apache Drill +version: +url: https://github.com/apache/drill-test-framework +license_category: source +module: java-core +license_name: Apache License version 2.0 +source_paths: + - sql/src/test/resources/drill/window + +--- + name: AWS SDK for Java license_category: binary module: java-core diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index 85aa73ad1d5..ca0bf0e1d6f 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -154,6 +154,28 @@ public class DruidException extends RuntimeException return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode())); } + /** + * Build a "defensive" exception, this is an exception that should never actually be triggered, but we are + * throwing it inside of a defensive check. + * + * @return A builder for a defensive exception. + */ + public static DruidExceptionBuilder defensive() + { + return forPersona(Persona.DEVELOPER).ofCategory(Category.DEFENSIVE); + } + + /** + * Build a "defensive" exception, this is an exception that should never actually be triggered, but we are + * throwing it inside of a defensive check. + * + * @return A builder for a defensive exception. + */ + public static DruidException defensive(String format, Object... args) + { + return defensive().build(format, args); + } + private final Persona targetPersona; private final Category category; private final String errorCode; @@ -340,7 +362,7 @@ public class DruidException extends RuntimeException UNSUPPORTED(501), /** * A catch-all for any time when we cannot come up with a meaningful categorization. This is hopefully only - * used when converting generic exceptions from frameworks and libraries that we do not control into DruidExcpetions + * used when converting generic exceptions from frameworks and libraries that we do not control into DruidExceptions */ UNCATEGORIZED(500); diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/ComplexFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/ComplexFrameColumnReader.java index e4a39c16f75..7059d76f391 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/ComplexFrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/ComplexFrameColumnReader.java @@ -27,6 +27,9 @@ import org.apache.druid.frame.write.columnar.FrameColumnWriters; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; +import org.apache.druid.query.rowsandcols.column.accessor.ObjectColumnAccessorBase; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.ObjectColumnSelector; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; @@ -36,7 +39,9 @@ import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; +import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.Comparator; public class ComplexFrameColumnReader implements FrameColumnReader { @@ -47,8 +52,28 @@ public class ComplexFrameColumnReader implements FrameColumnReader this.columnNumber = columnNumber; } + @Override + public Column readRACColumn(Frame frame) + { + return new ColumnAccessorBasedColumn(makeComplexFrameColumn(frame)); + } + @Override public ColumnPlus readColumn(final Frame frame) + { + final ComplexFrameColumn frameCol = makeComplexFrameColumn(frame); + + return new ColumnPlus( + frameCol, + new ColumnCapabilitiesImpl() + .setType(frameCol.getType()) + .setHasMultipleValues(false), + frame.numRows() + ); + } + + @Nonnull + private ComplexFrameColumn makeComplexFrameColumn(Frame frame) { final Memory memory = frame.region(columnNumber); validate(memory, frame.numRows()); @@ -68,17 +93,12 @@ public class ComplexFrameColumnReader implements FrameColumnReader final long startOfOffsetSection = Byte.BYTES + Integer.BYTES + typeNameLength; final long startOfDataSection = startOfOffsetSection + (long) frame.numRows() * Integer.BYTES; - return new ColumnPlus( - new ComplexFrameColumn( - frame, - serde, - memory, - startOfOffsetSection, - startOfDataSection - ), - new ColumnCapabilitiesImpl().setType(ColumnType.ofComplex(typeName)) - .setHasMultipleValues(false), - frame.numRows() + return new ComplexFrameColumn( + frame, + serde, + memory, + startOfOffsetSection, + startOfDataSection ); } @@ -100,7 +120,7 @@ public class ComplexFrameColumnReader implements FrameColumnReader } } - private static class ComplexFrameColumn implements ComplexColumn + private static class ComplexFrameColumn extends ObjectColumnAccessorBase implements ComplexColumn { private final Frame frame; private final ComplexMetricSerde serde; @@ -184,6 +204,30 @@ public class ComplexFrameColumnReader implements FrameColumnReader // Do nothing. } + @Override + public ColumnType getType() + { + return ColumnType.ofComplex(serde.getTypeName()); + } + + @Override + public int numRows() + { + return getLength(); + } + + @Override + protected Object getVal(int rowNum) + { + return getRowValue(rowNum); + } + + @Override + protected Comparator getComparator() + { + return serde.getTypeStrategy(); + } + @Nullable private Object getObjectForPhysicalRow(final int physicalRow) { diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/DoubleFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/DoubleFrameColumnReader.java index 8ac4b1d15d6..e2c3051a373 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/DoubleFrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/DoubleFrameColumnReader.java @@ -26,10 +26,12 @@ import org.apache.druid.frame.write.columnar.DoubleFrameMaker; import org.apache.druid.frame.write.columnar.FrameColumnWriters; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; +import org.apache.druid.query.rowsandcols.column.accessor.DoubleColumnAccessorBase; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.NumericColumn; import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector; @@ -37,6 +39,7 @@ import org.apache.druid.segment.vector.ReadableVectorInspector; import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.VectorValueSelector; +import javax.annotation.Nonnull; import javax.annotation.Nullable; public class DoubleFrameColumnReader implements FrameColumnReader @@ -48,20 +51,35 @@ public class DoubleFrameColumnReader implements FrameColumnReader this.columnNumber = columnNumber; } + @Override + public Column readRACColumn(Frame frame) + { + final DoubleFrameColumn frameCol = makeDoubleFrameColumn(frame); + + return new ColumnAccessorBasedColumn(frameCol); + } + @Override public ColumnPlus readColumn(final Frame frame) + { + final DoubleFrameColumn frameCol = makeDoubleFrameColumn(frame); + + return new ColumnPlus( + frameCol, + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(frameCol.getType()) + .setHasNulls(NullHandling.sqlCompatible() && frameCol.hasNulls), + frame.numRows() + ); + } + + @Nonnull + private DoubleFrameColumn makeDoubleFrameColumn(Frame frame) { final Memory memory = frame.region(columnNumber); validate(memory, frame.numRows()); final boolean hasNulls = getHasNulls(memory); - - return new ColumnPlus( - new DoubleFrameColumn(frame, hasNulls, memory), - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE) - .setHasNulls(NullHandling.sqlCompatible() && hasNulls), - frame.numRows() - ); + return new DoubleFrameColumn(frame, hasNulls, memory); } private void validate(final Memory region, final int numRows) @@ -92,7 +110,7 @@ public class DoubleFrameColumnReader implements FrameColumnReader return memory.getByte(Byte.BYTES) != 0; } - private static class DoubleFrameColumn implements NumericColumn + private static class DoubleFrameColumn extends DoubleColumnAccessorBase implements NumericColumn { private final Frame frame; private final boolean hasNulls; @@ -122,13 +140,13 @@ public class DoubleFrameColumnReader implements FrameColumnReader public double getDouble() { assert NullHandling.replaceWithDefault() || !isNull(); - return DoubleFrameColumn.this.getDouble(frame.physicalRow(offset.getOffset())); + return DoubleFrameColumn.this.getDoublePhysical(frame.physicalRow(offset.getOffset())); } @Override public boolean isNull() { - return DoubleFrameColumn.this.isNull(frame.physicalRow(offset.getOffset())); + return DoubleFrameColumn.this.isNullPhysical(frame.physicalRow(offset.getOffset())); } @Override @@ -182,10 +200,10 @@ public class DoubleFrameColumnReader implements FrameColumnReader for (int i = 0; i < offset.getCurrentVectorSize(); i++) { final int physicalRow = frame.physicalRow(i + start); - doubleVector[i] = getDouble(physicalRow); + doubleVector[i] = getDoublePhysical(physicalRow); if (hasNulls) { - nullVector[i] = isNull(physicalRow); + nullVector[i] = isNullPhysical(physicalRow); } } } else { @@ -193,10 +211,10 @@ public class DoubleFrameColumnReader implements FrameColumnReader for (int i = 0; i < offset.getCurrentVectorSize(); i++) { final int physicalRow = frame.physicalRow(offsets[i]); - doubleVector[i] = getDouble(physicalRow); + doubleVector[i] = getDoublePhysical(physicalRow); if (hasNulls) { - nullVector[i] = isNull(physicalRow); + nullVector[i] = isNullPhysical(physicalRow); } } } @@ -222,7 +240,7 @@ public class DoubleFrameColumnReader implements FrameColumnReader throw new ISE("Row [%d] out of bounds", rowNum); } - return (long) getDouble(frame.physicalRow(rowNum)); + return (long) getDoublePhysical(frame.physicalRow(rowNum)); } @Override @@ -237,7 +255,25 @@ public class DoubleFrameColumnReader implements FrameColumnReader // Do nothing. } - private boolean isNull(final int physicalRow) + @Override + public int numRows() + { + return length(); + } + + @Override + public boolean isNull(int rowNum) + { + return isNullPhysical(frame.physicalRow(rowNum)); + } + + @Override + public double getDouble(int rowNum) + { + return getDoublePhysical(frame.physicalRow(rowNum)); + } + + private boolean isNullPhysical(final int physicalRow) { if (hasNulls) { final long rowPosition = memoryPosition + (long) sz * physicalRow; @@ -247,7 +283,7 @@ public class DoubleFrameColumnReader implements FrameColumnReader } } - private double getDouble(final int physicalRow) + private double getDoublePhysical(final int physicalRow) { final long rowPosition = memoryPosition + (long) sz * physicalRow; diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/FloatFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/FloatFrameColumnReader.java index 0f1d09f57bc..31c0f85f956 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/FloatFrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/FloatFrameColumnReader.java @@ -26,10 +26,12 @@ import org.apache.druid.frame.write.columnar.FloatFrameMaker; import org.apache.druid.frame.write.columnar.FrameColumnWriters; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; +import org.apache.druid.query.rowsandcols.column.accessor.FloatColumnAccessorBase; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.FloatColumnSelector; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.NumericColumn; import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.vector.BaseFloatVectorValueSelector; @@ -37,6 +39,7 @@ import org.apache.druid.segment.vector.ReadableVectorInspector; import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.VectorValueSelector; +import javax.annotation.Nonnull; import javax.annotation.Nullable; public class FloatFrameColumnReader implements FrameColumnReader @@ -48,20 +51,35 @@ public class FloatFrameColumnReader implements FrameColumnReader this.columnNumber = columnNumber; } + @Override + public Column readRACColumn(Frame frame) + { + final FloatFrameColumn frameCol = makeFloatFrameColumn(frame); + + return new ColumnAccessorBasedColumn(frameCol); + } + @Override public ColumnPlus readColumn(final Frame frame) + { + final FloatFrameColumn frameCol = makeFloatFrameColumn(frame); + + return new ColumnPlus( + frameCol, + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(frameCol.getType()) + .setHasNulls(NullHandling.sqlCompatible() && frameCol.hasNulls), + frame.numRows() + ); + } + + @Nonnull + private FloatFrameColumn makeFloatFrameColumn(Frame frame) { final Memory memory = frame.region(columnNumber); validate(memory, frame.numRows()); final boolean hasNulls = getHasNulls(memory); - - return new ColumnPlus( - new FloatFrameColumn(frame, hasNulls, memory), - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT) - .setHasNulls(NullHandling.sqlCompatible() && hasNulls), - frame.numRows() - ); + return new FloatFrameColumn(frame, hasNulls, memory); } private void validate(final Memory region, final int numRows) @@ -92,7 +110,7 @@ public class FloatFrameColumnReader implements FrameColumnReader return memory.getByte(Byte.BYTES) != 0; } - private static class FloatFrameColumn implements NumericColumn + private static class FloatFrameColumn extends FloatColumnAccessorBase implements NumericColumn { private final Frame frame; private final boolean hasNulls; @@ -122,13 +140,13 @@ public class FloatFrameColumnReader implements FrameColumnReader public float getFloat() { assert NullHandling.replaceWithDefault() || !isNull(); - return FloatFrameColumn.this.getFloat(frame.physicalRow(offset.getOffset())); + return FloatFrameColumn.this.getFloatInternal(frame.physicalRow(offset.getOffset())); } @Override public boolean isNull() { - return FloatFrameColumn.this.isNull(frame.physicalRow(offset.getOffset())); + return FloatFrameColumn.this.isNullInternal(frame.physicalRow(offset.getOffset())); } @Override @@ -182,10 +200,10 @@ public class FloatFrameColumnReader implements FrameColumnReader for (int i = 0; i < offset.getCurrentVectorSize(); i++) { final int physicalRow = frame.physicalRow(i + start); - floatVector[i] = getFloat(physicalRow); + floatVector[i] = getFloatInternal(physicalRow); if (hasNulls) { - nullVector[i] = isNull(physicalRow); + nullVector[i] = isNullInternal(physicalRow); } } } else { @@ -193,10 +211,10 @@ public class FloatFrameColumnReader implements FrameColumnReader for (int i = 0; i < offset.getCurrentVectorSize(); i++) { final int physicalRow = frame.physicalRow(offsets[i]); - floatVector[i] = getFloat(physicalRow); + floatVector[i] = getFloatInternal(physicalRow); if (hasNulls) { - nullVector[i] = isNull(physicalRow); + nullVector[i] = isNullInternal(physicalRow); } } } @@ -222,7 +240,7 @@ public class FloatFrameColumnReader implements FrameColumnReader throw new ISE("Row [%d] out of bounds", rowNum); } - return (long) getFloat(frame.physicalRow(rowNum)); + return (long) getFloatInternal(frame.physicalRow(rowNum)); } @Override @@ -237,7 +255,25 @@ public class FloatFrameColumnReader implements FrameColumnReader // Do nothing. } - private boolean isNull(final int physicalRow) + @Override + public int numRows() + { + return length(); + } + + @Override + public boolean isNull(int rowNum) + { + return isNullInternal(frame.physicalRow(rowNum)); + } + + @Override + public float getFloat(int rowNum) + { + return getFloatInternal(frame.physicalRow(rowNum)); + } + + private boolean isNullInternal(final int physicalRow) { if (hasNulls) { final long rowPosition = memoryPosition + (long) sz * physicalRow; @@ -247,7 +283,7 @@ public class FloatFrameColumnReader implements FrameColumnReader } } - private float getFloat(final int physicalRow) + private float getFloatInternal(final int physicalRow) { final long rowPosition = memoryPosition + (long) sz * physicalRow; diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReader.java index 21735aa990b..58f1c10e72b 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReader.java @@ -20,6 +20,7 @@ package org.apache.druid.frame.read.columnar; import org.apache.druid.frame.Frame; +import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.segment.column.RowSignature; /** @@ -28,6 +29,11 @@ import org.apache.druid.segment.column.RowSignature; */ public interface FrameColumnReader { + /** + * Returns a {@link Column} from the frame. + */ + Column readRACColumn(Frame frame); + /** * Returns a column reference for the provided frame. */ diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/LongFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/LongFrameColumnReader.java index 1662411b912..8bfc69047f4 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/LongFrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/LongFrameColumnReader.java @@ -26,10 +26,12 @@ import org.apache.druid.frame.write.columnar.FrameColumnWriters; import org.apache.druid.frame.write.columnar.LongFrameMaker; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; +import org.apache.druid.query.rowsandcols.column.accessor.LongColumnAccessorBase; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.NumericColumn; import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.vector.BaseLongVectorValueSelector; @@ -37,6 +39,7 @@ import org.apache.druid.segment.vector.ReadableVectorInspector; import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.VectorValueSelector; +import javax.annotation.Nonnull; import javax.annotation.Nullable; public class LongFrameColumnReader implements FrameColumnReader @@ -48,20 +51,36 @@ public class LongFrameColumnReader implements FrameColumnReader this.columnNumber = columnNumber; } + @Override + public Column readRACColumn(Frame frame) + { + final LongFrameColumn frameCol = makeLongFrameColumn(frame); + + return new ColumnAccessorBasedColumn(frameCol); + + } + @Override public ColumnPlus readColumn(final Frame frame) + { + final LongFrameColumn frameCol = makeLongFrameColumn(frame); + + return new ColumnPlus( + frameCol, + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(frameCol.getType()) + .setHasNulls(NullHandling.sqlCompatible() && frameCol.hasNulls), + frame.numRows() + ); + } + + @Nonnull + private LongFrameColumn makeLongFrameColumn(Frame frame) { final Memory memory = frame.region(columnNumber); validate(memory, frame.numRows()); final boolean hasNulls = getHasNulls(memory); - - return new ColumnPlus( - new LongFrameColumn(frame, hasNulls, memory), - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG) - .setHasNulls(NullHandling.sqlCompatible() && hasNulls), - frame.numRows() - ); + return new LongFrameColumn(frame, hasNulls, memory); } private void validate(final Memory region, final int numRows) @@ -90,7 +109,7 @@ public class LongFrameColumnReader implements FrameColumnReader return memoryRange.getByte(Byte.BYTES) != 0; } - private static class LongFrameColumn implements NumericColumn + private static class LongFrameColumn extends LongColumnAccessorBase implements NumericColumn { private final Frame frame; private final boolean hasNulls; @@ -120,13 +139,13 @@ public class LongFrameColumnReader implements FrameColumnReader public long getLong() { assert NullHandling.replaceWithDefault() || !isNull(); - return LongFrameColumn.this.getLong(frame.physicalRow(offset.getOffset())); + return LongFrameColumn.this.getLongPhysical(frame.physicalRow(offset.getOffset())); } @Override public boolean isNull() { - return LongFrameColumn.this.isNull(frame.physicalRow(offset.getOffset())); + return LongFrameColumn.this.isNullPhysical(frame.physicalRow(offset.getOffset())); } @Override @@ -180,10 +199,10 @@ public class LongFrameColumnReader implements FrameColumnReader for (int i = 0; i < offset.getCurrentVectorSize(); i++) { final int physicalRow = frame.physicalRow(i + start); - longVector[i] = getLong(physicalRow); + longVector[i] = getLongPhysical(physicalRow); if (hasNulls) { - nullVector[i] = isNull(physicalRow); + nullVector[i] = isNullPhysical(physicalRow); } } } else { @@ -191,10 +210,10 @@ public class LongFrameColumnReader implements FrameColumnReader for (int i = 0; i < offset.getCurrentVectorSize(); i++) { final int physicalRow = frame.physicalRow(offsets[i]); - longVector[i] = getLong(physicalRow); + longVector[i] = getLongPhysical(physicalRow); if (hasNulls) { - nullVector[i] = isNull(physicalRow); + nullVector[i] = isNullPhysical(physicalRow); } } } @@ -220,7 +239,7 @@ public class LongFrameColumnReader implements FrameColumnReader throw new ISE("Row [%d] out of bounds", rowNum); } - return getLong(frame.physicalRow(rowNum)); + return getLongPhysical(frame.physicalRow(rowNum)); } @Override @@ -235,7 +254,25 @@ public class LongFrameColumnReader implements FrameColumnReader // Do nothing. } - private boolean isNull(final int physicalRow) + @Override + public int numRows() + { + return length(); + } + + @Override + public boolean isNull(int rowNum) + { + return isNullPhysical(frame.physicalRow(rowNum)); + } + + @Override + public long getLong(int rowNum) + { + return getLongPhysical(frame.physicalRow(rowNum)); + } + + private boolean isNullPhysical(final int physicalRow) { if (hasNulls) { final long rowPosition = memoryPosition + (long) sz * physicalRow; @@ -245,7 +282,7 @@ public class LongFrameColumnReader implements FrameColumnReader } } - private long getLong(final int physicalRow) + private long getLongPhysical(final int physicalRow) { final long rowPosition = memoryPosition + (long) sz * physicalRow; diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java index 706b72432ca..2193d6a23e1 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java @@ -34,6 +34,9 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; +import org.apache.druid.query.rowsandcols.column.accessor.ObjectColumnAccessorBase; import org.apache.druid.segment.BaseSingleValueDimensionSelector; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionDictionarySelector; @@ -59,6 +62,7 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.List; public class StringFrameColumnReader implements FrameColumnReader @@ -72,6 +76,27 @@ public class StringFrameColumnReader implements FrameColumnReader this.asArray = asArray; } + @Override + public Column readRACColumn(Frame frame) + { + final Memory memory = frame.region(columnNumber); + validate(memory); + + if (isMultiValue(memory)) { + // When we implement handling of multi-value, we should actually make this look like an Array of String instead + // of perpetuating the multi-value idea. Thus, when we add support for Arrays to the RAC stuff, that's when + // we can start supporting multi-value. + throw new ISE("Multivalue not yet handled by RAC"); + } + final long positionOfLengths = getStartOfStringLengthSection(frame.numRows(), false); + final long positionOfPayloads = getStartOfStringDataSection(memory, frame.numRows(), false); + + StringFrameColumn frameCol = new StringFrameColumn(frame, false, memory, positionOfLengths, positionOfPayloads); + + return new ColumnAccessorBasedColumn(frameCol); + + } + @Override public ColumnPlus readColumn(final Frame frame) { @@ -171,7 +196,7 @@ public class StringFrameColumnReader implements FrameColumnReader } @VisibleForTesting - static class StringFrameColumn implements DictionaryEncodedColumn + static class StringFrameColumn extends ObjectColumnAccessorBase implements DictionaryEncodedColumn { private final Frame frame; private final boolean multiValue; @@ -459,6 +484,30 @@ public class StringFrameColumnReader implements FrameColumnReader // Do nothing. } + @Override + public ColumnType getType() + { + return ColumnType.STRING; + } + + @Override + public int numRows() + { + return length(); + } + + @Override + protected Object getVal(int rowNum) + { + return getString(frame.physicalRow(rowNum)); + } + + @Override + protected Comparator getComparator() + { + return Comparator.nullsFirst(Comparator.comparing(o -> ((String) o))); + } + /** * Returns a ByteBuffer containing UTF-8 encoded string number {@code index}. The ByteBuffer is always newly * created, so it is OK to change its position, limit, etc. However, it may point to shared memory, so it is diff --git a/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java b/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java index ddbe253de01..30cc388f1d9 100644 --- a/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java +++ b/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java @@ -36,6 +36,8 @@ import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.FrameBasedInlineDataSourceSerializer; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.context.ResponseContextDeserializer; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.WireTransferable; import org.joda.time.DateTimeZone; import java.io.IOException; @@ -186,5 +188,21 @@ public class DruidDefaultSerializersModule extends SimpleModule } ); addDeserializer(ResponseContext.class, new ResponseContextDeserializer()); + + addSerializer(RowsAndColumns.class, new JsonSerializer() + { + @Override + public void serialize( + RowsAndColumns value, + JsonGenerator gen, + SerializerProvider serializers + ) throws IOException + { + // It would be really cool if jackson offered an output stream that would allow us to push bytes + // through, but it doesn't right now, so we have to build a byte[] instead. Maybe something to contribute + // back to Jackson at some point. + gen.writeBinary(WireTransferable.fromRAC(value).bytesToTransfer()); + } + }); } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/LimitTimeIntervalOperator.java b/processing/src/main/java/org/apache/druid/query/operator/LimitTimeIntervalOperator.java new file mode 100644 index 00000000000..b658335e6a9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/LimitTimeIntervalOperator.java @@ -0,0 +1,78 @@ +/* + * 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.druid.query.operator; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.RowsAndColumnsDecorator; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.util.List; + +public class LimitTimeIntervalOperator implements Operator +{ + private final Operator segmentOperator; + private Interval interval; + + public LimitTimeIntervalOperator( + Operator segmentOperator, + QueryPlus queryPlus + ) + { + this.segmentOperator = segmentOperator; + + final List intervals = queryPlus.getQuery().getIntervals(); + if (intervals.size() != 1) { + throw new ISE("Can only handle a single interval, got[%s]", intervals); + } + interval = intervals.get(0); + } + + @Nullable + @Override + public Closeable goOrContinue( + Closeable continuationObject, + Receiver receiver + ) + { + return segmentOperator.goOrContinue(continuationObject, new Receiver() + { + @Override + public Signal push(RowsAndColumns rac) + { + final RowsAndColumnsDecorator decor = RowsAndColumnsDecorator.fromRAC(rac); + if (!Intervals.isEternity(interval)) { + decor.limitTimeRange(interval); + } + return receiver.push(decor.toRowsAndColumns()); + } + + @Override + public void completed() + { + receiver.completed(); + } + }); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index 90000a9fa6a..0c68c3eea27 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -19,13 +19,17 @@ package org.apache.druid.query.operator; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.RE; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; import java.io.Closeable; +import java.io.IOException; import java.util.Iterator; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; /** * This naive partitioning operator assumes that it's child operator always gives it RowsAndColumns objects that are @@ -41,8 +45,6 @@ public class NaivePartitioningOperator implements Operator private final List partitionColumns; private final Operator child; - private Iterator partitionsIter; - public NaivePartitioningOperator( List partitionColumns, Operator child @@ -55,33 +57,120 @@ public class NaivePartitioningOperator implements Operator @Override public Closeable goOrContinue(Closeable continuation, Receiver receiver) { - return child.goOrContinue( + if (continuation != null) { + Continuation cont = (Continuation) continuation; + + if (cont.iter != null) { + while (cont.iter.hasNext()) { + final Signal signal = receiver.push(cont.iter.next()); + switch (signal) { + case GO: + break; + case PAUSE: + if (cont.iter.hasNext()) { + return cont; + } + + if (cont.subContinuation == null) { + // We were finished anyway + receiver.completed(); + return null; + } + + return new Continuation(null, cont.subContinuation); + case STOP: + receiver.completed(); + try { + cont.close(); + } + catch (IOException e) { + throw new RE(e, "Unable to close continutation"); + } + return null; + default: + throw new RE("Unknown signal[%s]", signal); + } + } + + if (cont.subContinuation == null) { + receiver.completed(); + return null; + } + } + + continuation = cont.subContinuation; + } + + AtomicReference> iterHolder = new AtomicReference<>(); + + final Closeable retVal = child.goOrContinue( continuation, new Receiver() { @Override public Signal push(RowsAndColumns rac) { + if (rac == null) { + throw DruidException.defensive("Should never get a null rac here."); + } ClusteredGroupPartitioner groupPartitioner = rac.as(ClusteredGroupPartitioner.class); if (groupPartitioner == null) { groupPartitioner = new DefaultClusteredGroupPartitioner(rac); } - partitionsIter = groupPartitioner.partitionOnBoundaries(partitionColumns).iterator(); + Iterator partitionsIter = + groupPartitioner.partitionOnBoundaries(partitionColumns).iterator(); Signal keepItGoing = Signal.GO; while (keepItGoing == Signal.GO && partitionsIter.hasNext()) { keepItGoing = receiver.push(partitionsIter.next()); } + + if (keepItGoing == Signal.PAUSE && partitionsIter.hasNext()) { + iterHolder.set(partitionsIter); + return Signal.PAUSE; + } + return keepItGoing; } @Override public void completed() { - receiver.completed(); + if (iterHolder.get() == null) { + receiver.completed(); + } } } ); + + if (iterHolder.get() != null || retVal != null) { + return new Continuation( + iterHolder.get(), + retVal + ); + } else { + return null; + } + } + + private static class Continuation implements Closeable + { + Iterator iter; + Closeable subContinuation; + + public Continuation(Iterator iter, Closeable subContinuation) + { + this.iter = iter; + this.subContinuation = subContinuation; + } + + @Override + public void close() throws IOException + { + if (subContinuation != null) { + subContinuation.close(); + } + } } } diff --git a/processing/src/main/java/org/apache/druid/query/operator/OperatorSequence.java b/processing/src/main/java/org/apache/druid/query/operator/OperatorSequence.java index 9fcec5d5525..9bb90189cdb 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/OperatorSequence.java +++ b/processing/src/main/java/org/apache/druid/query/operator/OperatorSequence.java @@ -91,6 +91,7 @@ public class OperatorSequence implements Sequence @Override public Yielder next(OutType initValue) { + accumulator.reset(); if (continuation == null) { // This means that we completed processing on the previous run. In this case, we are all done return Yielders.done(null, null); diff --git a/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java b/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java new file mode 100644 index 00000000000..b823c30d22e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java @@ -0,0 +1,118 @@ +/* + * 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.druid.query.operator; + +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.RowsAndColumnsDecorator; +import org.apache.druid.segment.VirtualColumns; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.util.List; + +/** + * A scan operator is really just a way to push down various things that can be lazily applied when data needs to + * actually be read. This ScanOperator works by converting the RowsAndColumns to a DecoratableRowsAndColumns + * and then just decorates it with metadata, in this way, the RowsAndColumns should be able to leverage the extra + * metadata whenever it is asked to do something else with a different Semantic interface by the next Operator. + *

    + * If it is important to materialize the data after this metadata is applied, the Projectable semantic interface + * can be used to materialize a new RowsAndColumns. + */ +public class ScanOperator implements Operator +{ + private final Operator subOperator; + private final Interval timeRange; + private final Filter filter; + private final int limit; + private final List projectedColumns; + private final VirtualColumns virtualColumns; + private final List ordering; + + public ScanOperator( + Operator subOperator, + List projectedColumns, + VirtualColumns virtualColumns, + Interval timeRange, + Filter filter, + List ordering, + int limit + ) + { + this.subOperator = subOperator; + this.projectedColumns = projectedColumns; + this.virtualColumns = virtualColumns; + this.timeRange = timeRange; + this.filter = filter; + this.ordering = ordering; + this.limit = limit; + } + + @Nullable + @Override + public Closeable goOrContinue( + Closeable continuationObject, + Receiver receiver + ) + { + return subOperator.goOrContinue(continuationObject, new Receiver() + { + @Override + public Signal push(RowsAndColumns rac) + { + final RowsAndColumnsDecorator decor = RowsAndColumnsDecorator.fromRAC(rac); + + if (filter != null) { + decor.addFilter(filter); + } + + if (virtualColumns != null) { + decor.addVirtualColumns(virtualColumns); + } + + if (timeRange != null) { + decor.limitTimeRange(timeRange); + } + + if (limit > 0) { + decor.setLimit(limit); + } + + if (!(ordering == null || ordering.isEmpty())) { + decor.setOrdering(ordering); + } + + if (!(projectedColumns == null || projectedColumns.isEmpty())) { + return receiver.push(decor.restrictColumns(projectedColumns)); + } else { + return receiver.push(decor.toRowsAndColumns()); + } + } + + @Override + public void completed() + { + receiver.completed(); + } + }); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java new file mode 100644 index 00000000000..a764984855e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java @@ -0,0 +1,136 @@ +/* + * 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.druid.query.operator; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.segment.VirtualColumns; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Objects; + +public class ScanOperatorFactory implements OperatorFactory +{ + private final Interval timeRange; + private final DimFilter filter; + private final int limit; + private final List projectedColumns; + private final VirtualColumns virtualColumns; + private final List ordering; + + public ScanOperatorFactory( + @JsonProperty("timeRange") final Interval timeRange, + @JsonProperty("filter") final DimFilter filter, + @JsonProperty("limit") final Integer limit, + @JsonProperty("projectedColumns") final List projectedColumns, + @JsonProperty("virtualColumns") final VirtualColumns virtualColumns, + @JsonProperty("ordering") final List ordering + ) + { + this.timeRange = timeRange; + this.filter = filter; + this.limit = limit == null ? -1 : limit; + this.projectedColumns = projectedColumns; + this.virtualColumns = virtualColumns; + this.ordering = ordering; + } + + @JsonProperty + public Interval getTimeRange() + { + return timeRange; + } + + @JsonProperty + public DimFilter getFilter() + { + return filter; + } + + @JsonProperty + public int getLimit() + { + return limit; + } + + @JsonProperty + public List getProjectedColumns() + { + return projectedColumns; + } + + @JsonProperty + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + + @JsonProperty + public List getOrdering() + { + return ordering; + } + + @Override + public Operator wrap(Operator op) + { + return new ScanOperator( + op, + projectedColumns, + virtualColumns, + timeRange, + filter == null ? null : filter.toFilter(), + ordering, + limit + ); + } + + @Override + public boolean validateEquivalent(OperatorFactory other) + { + return equals(other); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof ScanOperatorFactory)) { + return false; + } + ScanOperatorFactory that = (ScanOperatorFactory) o; + return limit == that.limit && Objects.equals(timeRange, that.timeRange) && Objects.equals( + filter, + that.filter + ) && Objects.equals(projectedColumns, that.projectedColumns) && Objects.equals( + virtualColumns, + that.virtualColumns + ) && Objects.equals(ordering, that.ordering); + } + + @Override + public int hashCode() + { + return Objects.hash(timeRange, filter, limit, projectedColumns, virtualColumns, ordering); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/SegmentToRowsAndColumnsOperator.java b/processing/src/main/java/org/apache/druid/query/operator/SegmentToRowsAndColumnsOperator.java index e60589497d8..7155f655636 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/SegmentToRowsAndColumnsOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/SegmentToRowsAndColumnsOperator.java @@ -46,8 +46,13 @@ public class SegmentToRowsAndColumnsOperator implements Operator if (shifty == null) { throw new ISE("Segment[%s] cannot shapeshift", segment.getClass()); } + RowsAndColumns rac; + if (shifty instanceof RowsAndColumns) { + rac = (RowsAndColumns) shifty; + } else { + rac = shifty.as(RowsAndColumns.class); + } - RowsAndColumns rac = shifty.as(RowsAndColumns.class); if (rac == null) { throw new ISE("Cannot work with segment of type[%s]", segment.getClass()); } diff --git a/processing/src/main/java/org/apache/druid/query/operator/SequenceOperator.java b/processing/src/main/java/org/apache/druid/query/operator/SequenceOperator.java index 7f42a0dbf2b..bbaa2851760 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/SequenceOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/SequenceOperator.java @@ -22,7 +22,7 @@ package org.apache.druid.query.operator; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.YieldingAccumulator; +import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.rowsandcols.RowsAndColumns; @@ -46,92 +46,72 @@ public class SequenceOperator implements Operator @Override public Closeable goOrContinue(Closeable continuation, Receiver receiver) { - Yielder yielder = null; - final Signal theSignal; - if (continuation == null) { - yielder = child.toYielder(Signal.GO, new YieldingAccumulator() - { - @Override - public Signal accumulate(Signal accumulated, RowsAndColumns in) - { - final Signal pushSignal = receiver.push(in); - switch (pushSignal) { - case PAUSE: - this.yield(); - return Signal.PAUSE; - case GO: - return Signal.GO; - case STOP: - this.yield(); - return Signal.STOP; - default: - throw new ISE("How can this be happening? signal[%s]", pushSignal); + Yielder yielder = null; + try { + if (continuation == null) { + yielder = Yielders.each(child); + } else { + yielder = (Yielder) continuation; + } + + while (true) { + Signal signal; + if (yielder.isDone()) { + if (yielder.get() != null) { + throw new ISE("Got a non-null get()[%s] even though we were already done.", yielder.get().getClass()); + } + signal = Signal.STOP; + } else { + signal = receiver.push(yielder.get()); + } + + if (signal != Signal.STOP) { + yielder = yielder.next(null); + if (yielder.isDone()) { + signal = Signal.STOP; } } - }); - theSignal = yielder.get(); - } else { + + switch (signal) { + case STOP: + receiver.completed(); + + try { + yielder.close(); + } + catch (IOException e) { + // We got an exception when closing after we received a STOP signal and successfully called completed(). + // This means that the Receiver has already done what it needs, so instead of throw the exception and + // potentially impact processing, we log instead and allow processing to continue. + log.warn( + e, + "Exception thrown when closing yielder. Logging and ignoring because results should be fine." + ); + } + return null; + case PAUSE: + return yielder; + case GO: + continue; + default: + throw new ISE("Unknown signal[%s]", signal); + } + } + } + catch (RuntimeException re) { try { - final Yielder castedYielder = (Yielder) continuation; - if (castedYielder.isDone()) { - throw new ISE( - "The yielder is done! The previous go call should've resulted in completion instead of continuation" - ); - } - yielder = castedYielder.next(Signal.GO); - theSignal = yielder.get(); - } - catch (ClassCastException e) { - try { - if (yielder == null) { - // Got the exception when casting the continuation, close the continuation and move on. - continuation.close(); - } else { - // Got the exception when reading the result from the continuation, close the yielder and move on. - yielder.close(); - } - } - catch (IOException ex) { - e.addSuppressed( - new ISE("Unable to close continuation[%s] of type[%s]", continuation, continuation.getClass()) - ); - } - throw e; - } - } - - switch (theSignal) { - // We get GO from the yielder if the last push created a GO and there was nothing left in the sequence. - // I.e. we are done - case GO: - case STOP: - try { - receiver.completed(); - } - catch (RuntimeException e) { - try { - yielder.close(); - } - catch (IOException ioException) { - e.addSuppressed(ioException); - throw e; - } - } - - try { + if (yielder != null) { yielder.close(); + } else if (continuation != null) { + // The yielder will be non-null in most cases, this can likely only happen if the continuation that we + // received was not able to be cast to a yielder. + continuation.close(); } - catch (IOException e) { - // We got an exception when closing after we received a STOP signal and successfully called completed(). - // This means that the Receiver has already done what it needs, so instead of throw the exception and - // potentially impact processing, we log instead and allow processing to continue. - log.warn(e, "Exception thrown when closing yielder. Logging and ignoring because results should be fine."); - } - return null; - case PAUSE: - return yielder; + } + catch (IOException ioEx) { + re.addSuppressed(ioEx); + } + throw re; } - throw new ISE("How can this happen!? signal[%s]", theSignal); } - } diff --git a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java index e26620be0a8..a07bad0d082 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java +++ b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java @@ -22,8 +22,6 @@ package org.apache.druid.query.operator; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; import org.apache.druid.query.InlineDataSource; @@ -31,11 +29,12 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.rowsandcols.RowsAndColumns; -import org.apache.druid.query.spec.LegacySegmentSpec; +import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -52,27 +51,92 @@ import java.util.Objects; */ public class WindowOperatorQuery extends BaseQuery { + private static DataSource validateMaybeRewriteDataSource(DataSource dataSource, boolean hasLeafs) + { + if (hasLeafs) { + return dataSource; + } + + // We can re-write scan-style sub queries into an operator instead of doing the actual Scan query. So, we + // check for that and, if we are going to do the rewrite, then we return the sub datasource such that the + // parent constructor in BaseQuery stores the actual data source that we want to be distributed to. + + // At this point, we could also reach into a QueryDataSource and validate that the ordering expected by the + // partitioning at least aligns with the ordering coming from the underlying query. We unfortunately don't + // have enough information to validate that the underlying ordering aligns with expectations for the actual + // window operator queries, but maybe we could get that and validate it here too. + if (dataSource instanceof QueryDataSource) { + final Query subQuery = ((QueryDataSource) dataSource).getQuery(); + if (subQuery instanceof ScanQuery) { + return subQuery.getDataSource(); + } + return dataSource; + } else if (dataSource instanceof InlineDataSource) { + return dataSource; + } else { + throw new IAE("WindowOperatorQuery must run on top of a query or inline data source, got [%s]", dataSource); + } + } + private final RowSignature rowSignature; private final List operators; + private final List leafOperators; @JsonCreator public WindowOperatorQuery( @JsonProperty("dataSource") DataSource dataSource, + @JsonProperty("intervals") QuerySegmentSpec intervals, @JsonProperty("context") Map context, @JsonProperty("outputSignature") RowSignature rowSignature, - @JsonProperty("operatorDefinition") List operators + @JsonProperty("operatorDefinition") List operators, + @JsonProperty("leafOperators") List leafOperators ) { - super(dataSource, new LegacySegmentSpec(Intervals.ETERNITY), false, context); + super( + validateMaybeRewriteDataSource(dataSource, leafOperators != null), + intervals, + false, + context + ); this.rowSignature = rowSignature; this.operators = operators; - // At this point, we can also reach into a QueryDataSource and validate that the ordering expected by the - // partitioning at least aligns with the ordering coming from the underlying query. We unfortunately don't - // have enough information to validate that the underlying ordering aligns with expectations for the actual - // window operator queries, but maybe we could get that and validate it here too. - if (!(dataSource instanceof QueryDataSource || dataSource instanceof InlineDataSource)) { - throw new IAE("WindowOperatorQuery must run on top of a query or inline data source, got [%s]", dataSource); + if (leafOperators == null) { + this.leafOperators = new ArrayList<>(); + // We have to double check again because this was validated in a static context before passing to the `super()` + // and we cannot save state from that... Ah well. + + if (dataSource instanceof QueryDataSource) { + final Query subQuery = ((QueryDataSource) dataSource).getQuery(); + if (subQuery instanceof ScanQuery) { + ScanQuery scan = (ScanQuery) subQuery; + + ArrayList ordering = new ArrayList<>(); + for (ScanQuery.OrderBy orderBy : scan.getOrderBys()) { + ordering.add( + new ColumnWithDirection( + orderBy.getColumnName(), + ScanQuery.Order.DESCENDING == orderBy.getOrder() + ? ColumnWithDirection.Direction.DESC + : ColumnWithDirection.Direction.ASC + ) + ); + } + + this.leafOperators.add( + new ScanOperatorFactory( + null, + scan.getFilter(), + (int) scan.getScanRowsLimit(), + scan.getColumns(), + scan.getVirtualColumns(), + ordering + ) + ); + } + } + } else { + this.leafOperators = leafOperators; } } @@ -82,6 +146,12 @@ public class WindowOperatorQuery extends BaseQuery return operators; } + @JsonProperty("leafOperators") + public List getLeafOperators() + { + return leafOperators; + } + @JsonProperty("outputSignature") public RowSignature getRowSignature() { @@ -107,21 +177,31 @@ public class WindowOperatorQuery extends BaseQuery return Query.WINDOW_OPERATOR; } + @Override public Query withOverriddenContext(Map contextOverride) { return new WindowOperatorQuery( getDataSource(), + getQuerySegmentSpec(), computeOverriddenContext(getContext(), contextOverride), rowSignature, - operators + operators, + leafOperators ); } @Override public Query withQuerySegmentSpec(QuerySegmentSpec spec) { - throw new UOE("Cannot override querySegmentSpec on window operator query. [%s]", spec); + return new WindowOperatorQuery( + getDataSource(), + getQuerySegmentSpec(), + getContext(), + rowSignature, + operators, + leafOperators + ); } @Override @@ -129,9 +209,11 @@ public class WindowOperatorQuery extends BaseQuery { return new WindowOperatorQuery( dataSource, + getQuerySegmentSpec(), getContext(), rowSignature, - operators + operators, + leafOperators ); } diff --git a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQueryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQueryQueryRunnerFactory.java index 78f64360761..d716524bb06 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQueryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQueryQueryRunnerFactory.java @@ -19,7 +19,9 @@ package org.apache.druid.query.operator; -import com.google.common.collect.Iterables; +import com.google.common.base.Function; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; @@ -27,6 +29,8 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.segment.Segment; +import javax.annotation.Nullable; + public class WindowOperatorQueryQueryRunnerFactory implements QueryRunnerFactory { public static final WindowOperatorQueryQueryToolChest TOOLCHEST = new WindowOperatorQueryQueryToolChest(); @@ -35,7 +39,14 @@ public class WindowOperatorQueryQueryRunnerFactory implements QueryRunnerFactory public QueryRunner createRunner(Segment segment) { return (queryPlus, responseContext) -> - new OperatorSequence(() -> new SegmentToRowsAndColumnsOperator(segment)); + new OperatorSequence(() -> { + Operator op = new SegmentToRowsAndColumnsOperator(segment); + op = new LimitTimeIntervalOperator(op, queryPlus); + for (OperatorFactory leaf : ((WindowOperatorQuery) queryPlus.getQuery()).getLeafOperators()) { + op = leaf.wrap(op); + } + return op; + }); } @Override @@ -44,7 +55,27 @@ public class WindowOperatorQueryQueryRunnerFactory implements QueryRunnerFactory Iterable> queryRunners ) { - return Iterables.getOnlyElement(queryRunners); + // This merge is extremely naive, there is no ordering being imposed over the data, nor is there any attempt + // to shrink the size of the data before pushing it across the wire. This code implementation is intended more + // to make this work for tests and less to work in production. That's why the WindowOperatorQuery forces + // a super-secrete context parameter to be set to actually allow it to run a query that pushes all the way down + // like this. When this gets fixed, we can remove that parameter. + return (queryPlus, responseContext) -> Sequences.concat( + Sequences.map( + Sequences.simple(queryRunners), + new Function, Sequence>() + { + @Nullable + @Override + public Sequence apply( + @Nullable QueryRunner input + ) + { + return input.run(queryPlus, responseContext); + } + } + ) + ); } @Override @@ -52,4 +83,5 @@ public class WindowOperatorQueryQueryRunnerFactory implements QueryRunnerFactory { return TOOLCHEST; } + } diff --git a/processing/src/main/java/org/apache/druid/query/operator/join/SortedInnerJoinOperator.java b/processing/src/main/java/org/apache/druid/query/operator/join/SortedInnerJoinOperator.java index 2468d79f782..19bad5c6ab7 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/join/SortedInnerJoinOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/join/SortedInnerJoinOperator.java @@ -57,7 +57,7 @@ public class SortedInnerJoinOperator implements Operator { private static final Logger log = new Logger(SortedInnerJoinOperator.class); - private final ArrayList parts; + private final List partDefns; private final JoinConfig config; public SortedInnerJoinOperator( @@ -65,11 +65,7 @@ public class SortedInnerJoinOperator implements Operator JoinConfig config ) { - this.parts = new ArrayList<>(partDefns.size()); - for (JoinPartDefn partDefn : partDefns) { - parts.add(new JoinPart(partDefn.getOp(), partDefn.getJoinFields(), partDefn.getProjectFields())); - } - + this.partDefns = partDefns; this.config = config; } @@ -78,7 +74,7 @@ public class SortedInnerJoinOperator implements Operator { JoinLogic joinLogic; if (continuation == null) { - joinLogic = new JoinLogic(config, parts); + joinLogic = new JoinLogic(config, partDefns); } else { joinLogic = (JoinLogic) continuation; } @@ -119,17 +115,41 @@ public class SortedInnerJoinOperator implements Operator private JoinLogic( JoinConfig config, - ArrayList joinParts + List joinParts ) { this.config = config; - this.joinParts = joinParts; + this.joinParts = new ArrayList<>(joinParts.size()); + for (JoinPartDefn partDefn : joinParts) { + this.joinParts.add(new JoinPart(partDefn.getOp(), partDefn.getJoinFields(), partDefn.getProjectFields())); + } setNextPositionToLoad(joinParts.size() - 1); } public void go(Receiver receiver) { + if (state == State.PAUSED) { + nextPositionToLoad = -1; + for (int i = joinParts.size() - 1; i >= 0; i--) { + final JoinPart joinPart = joinParts.get(i); + if (joinPart.needsData()) { + nextPositionToLoad = i; + break; + } + } + + if (nextPositionToLoad == -1) { + state = State.READY; + // This means that all of the parts have data, so let's process what we have and then let it flow into the + // normal processing loop. + process(receiver); + if (state == State.PAUSED) { + return; + } + } + } + while (state != State.COMPLETE) { final int position = nextPositionToLoad; final JoinPart joinPart = joinParts.get(position); @@ -179,21 +199,21 @@ public class SortedInnerJoinOperator implements Operator case PAUSED: return; case COMPLETE: - receiver.completed(); - - try { - close(); - } - catch (IOException e) { - log.warn("Problem closing a join part[%d], ignoring because we are done anyway.", position); - } - break; case NEEDS_DATA: break; default: throw new ISE("Unknown state[%s]", state); } } + + receiver.completed(); + + try { + close(); + } + catch (IOException e) { + log.warn("Problem closing stuff, ignoring because we are done anyway."); + } } /** @@ -365,11 +385,8 @@ public class SortedInnerJoinOperator implements Operator if (rowsToInclude[0].size() > config.getReleaseSize()) { // Incrementally push stuff out once we've collected this number of rows pushRows(receiver, rowsToInclude); - if (state == State.READY) { - // We have more to do, so reinitialize rowsToInclude - for (DruidIntList intList : rowsToInclude) { - intList.resetToSize(config.getBufferSize()); - } + for (DruidIntList intList : rowsToInclude) { + intList.resetToSize(config.getBufferSize()); } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java index 92c10fb8e8e..93604af64c3 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java @@ -31,8 +31,8 @@ import org.apache.druid.query.rowsandcols.column.ColumnValueSwapper; import org.apache.druid.query.rowsandcols.column.DefaultVectorCopier; import org.apache.druid.query.rowsandcols.column.LimitedColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; -import org.apache.druid.query.rowsandcols.column.ObjectColumnAccessorBase; import org.apache.druid.query.rowsandcols.column.VectorCopier; +import org.apache.druid.query.rowsandcols.column.accessor.ObjectColumnAccessorBase; import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java new file mode 100644 index 00000000000..e6ef061d715 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -0,0 +1,361 @@ +/* + * 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.druid.query.rowsandcols; + +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; +import org.apache.druid.frame.key.KeyColumn; +import org.apache.druid.frame.key.KeyOrder; +import org.apache.druid.frame.write.FrameWriter; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.query.rowsandcols.concrete.FrameRowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.ColumnSelectorFactoryMaker; +import org.apache.druid.query.rowsandcols.semantic.DefaultRowsAndColumnsDecorator; +import org.apache.druid.query.rowsandcols.semantic.RowsAndColumnsDecorator; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.RowSignature; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +public class LazilyDecoratedRowsAndColumns implements RowsAndColumns +{ + private RowsAndColumns base; + private Interval interval; + private Filter filter; + private VirtualColumns virtualColumns; + private int limit; + private LinkedHashSet viewableColumns; + private List ordering; + + public LazilyDecoratedRowsAndColumns( + RowsAndColumns base, + Interval interval, + Filter filter, + VirtualColumns virtualColumns, + int limit, + List ordering, + LinkedHashSet viewableColumns + ) + { + this.base = base; + this.interval = interval; + this.filter = filter; + this.virtualColumns = virtualColumns; + this.limit = limit; + this.ordering = ordering; + this.viewableColumns = viewableColumns; + } + + @Override + public Collection getColumnNames() + { + return viewableColumns == null ? base.getColumnNames() : viewableColumns; + } + + @Override + public int numRows() + { + maybeMaterialize(); + return base.numRows(); + } + + @Nullable + @Override + public Column findColumn(String name) + { + if (viewableColumns != null && !viewableColumns.contains(name)) { + return null; + } + + maybeMaterialize(); + return base.findColumn(name); + } + + @SuppressWarnings("unchecked") + @Nullable + @Override + public T as(Class clazz) + { + if (RowsAndColumnsDecorator.class.equals(clazz)) { + // If we don't have a projection defined, then it's safe to continue collecting more decorations as we + // can meaningfully merge them together. + if (viewableColumns == null || viewableColumns.isEmpty()) { + return (T) new DefaultRowsAndColumnsDecorator( + base, + interval, + filter, + virtualColumns, + limit, + ordering + ); + } else { + return (T) new DefaultRowsAndColumnsDecorator(this); + } + } + return null; + } + + private void maybeMaterialize() + { + if (!(interval == null && filter == null && limit == -1 && ordering == null)) { + materialize(); + } + } + + private void materialize() + { + if (ordering != null) { + throw new ISE("Cannot reorder[%s] scan data right now", ordering); + } + + final StorageAdapter as = base.as(StorageAdapter.class); + if (as == null) { + reset(naiveMaterialize(base)); + } else { + reset(materializeStorageAdapter(as)); + } + } + + private RowsAndColumns materializeStorageAdapter(StorageAdapter as) + { + final Sequence cursors = as.makeCursors( + filter, + interval == null ? Intervals.ETERNITY : interval, + virtualColumns, + Granularities.ALL, + false, + null + ); + + Collection cols = viewableColumns == null ? base.getColumnNames() : viewableColumns; + AtomicReference siggy = new AtomicReference<>(null); + + FrameWriter writer = cursors.accumulate(null, (accumulated, in) -> { + if (accumulated != null) { + // We should not get multiple cursors because we set the granularity to ALL. So, this should never + // actually happen, but it doesn't hurt us to defensive here, so we test against it. + throw new ISE("accumulated[%s] non-null, why did we get multiple cursors?", accumulated); + } + + int theLimit = limit == -1 ? Integer.MAX_VALUE : limit; + + final ColumnSelectorFactory columnSelectorFactory = in.getColumnSelectorFactory(); + final RowSignature.Builder sigBob = RowSignature.builder(); + + for (String col : cols) { + final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(col); + if (capabilities != null) { + sigBob.add(col, capabilities.toColumnType()); + } + } + final RowSignature signature = sigBob.build(); + siggy.set(signature); + + List sortColumns = new ArrayList<>(); + if (ordering != null) { + for (ColumnWithDirection columnWithDirection : ordering) { + final KeyOrder order; + + if (columnWithDirection.getDirection() == ColumnWithDirection.Direction.DESC) { + order = KeyOrder.DESCENDING; + } else { + order = KeyOrder.ASCENDING; + } + + sortColumns.add(new KeyColumn(columnWithDirection.getColumn(), order)); + } + } + + final FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory( + FrameType.COLUMNAR, + new ArenaMemoryAllocatorFactory(200 << 20), // 200 MB, because, why not? + signature, + sortColumns + ); + + final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactory); + while (!in.isDoneOrInterrupted()) { + frameWriter.addSelection(); + in.advance(); + if (--theLimit <= 0) { + break; + } + } + + return frameWriter; + }); + + if (writer == null) { + // This means that the accumulate was never called, which can only happen if we didn't have any cursors. + // We would only have zero cursors if we essentially didn't match anything, meaning that our RowsAndColumns + // should be completely empty. + return new EmptyRowsAndColumns(); + } else { + final byte[] bytes = writer.toByteArray(); + return new FrameRowsAndColumns(Frame.wrap(bytes), siggy.get()); + } + } + + private void reset(RowsAndColumns rac) + { + base = rac; + interval = null; + filter = null; + virtualColumns = null; + limit = -1; + viewableColumns = null; + ordering = null; + } + + private RowsAndColumns naiveMaterialize(RowsAndColumns rac) + { + final int numRows = rac.numRows(); + + BitSet rowsToSkip = null; + + if (interval != null) { + rowsToSkip = new BitSet(numRows); + + final Column racColumn = rac.findColumn("__time"); + if (racColumn == null) { + // The time column doesn't exist, this means that we have a null column. A null column when coerced into a + // long as is required by the time filter produces all 0s, so either 0 is included and matches all rows or + // it's not and we skip all rows. + if (!interval.contains(0)) { + return new EmptyRowsAndColumns(); + } + } else { + final ColumnAccessor accessor = racColumn.toAccessor(); + for (int i = 0; i < accessor.numRows(); ++i) { + rowsToSkip.set(i, !interval.contains(accessor.getLong(i))); + } + } + } + + AtomicInteger rowId = new AtomicInteger(0); + final ColumnSelectorFactoryMaker csfm = ColumnSelectorFactoryMaker.fromRAC(rac); + final ColumnSelectorFactory selectorFactory = csfm.make(rowId); + + if (filter != null) { + if (rowsToSkip == null) { + rowsToSkip = new BitSet(numRows); + } + + final ValueMatcher matcher = filter.makeMatcher(selectorFactory); + + for (; rowId.get() < numRows; rowId.incrementAndGet()) { + final int theId = rowId.get(); + if (rowsToSkip.get(theId)) { + continue; + } + + if (!matcher.matches()) { + rowsToSkip.set(theId); + } + } + } + + if (virtualColumns != null) { + throw new UOE("Cannot apply virtual columns [%s] with naive apply.", virtualColumns); + } + + ArrayList columnsToGenerate = new ArrayList<>(); + if (viewableColumns != null) { + columnsToGenerate.addAll(viewableColumns); + } else { + columnsToGenerate.addAll(rac.getColumnNames()); + // When/if we support virtual columns from here, we should auto-add them to the list here as well as they expand + // the implicit project when no projection is defined + } + + // There is all sorts of sub-optimal things in this code, but we just ignore them for now as it is difficult to + // optimally build frames for incremental data processing. In order to build the frame object here, we must first + // materialize everything in memory so that we know how long things are, such that we can allocate a big byte[] + // so that the Frame.wrap() call can be given just a big byte[] to do its reading from. + // + // It would be a bit better if we could just build the per-column byte[] and somehow re-generate a Frame from + // that. But it's also possible that this impedence mis-match is a function of using column-oriented frames and + // row-oriented frames are a much more friendly format. Anyway, this long comment is here because this exploration + // is being left as an exercise for the future. + + final RowSignature.Builder sigBob = RowSignature.builder(); + final ArenaMemoryAllocatorFactory memFactory = new ArenaMemoryAllocatorFactory(200 << 20); + + for (String column : columnsToGenerate) { + final Column racColumn = rac.findColumn(column); + if (racColumn == null) { + continue; + } + + sigBob.add(column, racColumn.toAccessor().getType()); + } + + final int limitedNumRows; + if (limit == -1) { + limitedNumRows = Integer.MAX_VALUE; + } else { + limitedNumRows = limit; + } + + final FrameWriter frameWriter = FrameWriters.makeFrameWriterFactory( + FrameType.COLUMNAR, + memFactory, + sigBob.build(), + Collections.emptyList() + ).newFrameWriter(selectorFactory); + + rowId.set(0); + for (; rowId.get() < numRows && frameWriter.getNumRows() < limitedNumRows; rowId.incrementAndGet()) { + final int theId = rowId.get(); + if (rowsToSkip != null && rowsToSkip.get(theId)) { + continue; + } + frameWriter.addSelection(); + } + + return new FrameRowsAndColumns(Frame.wrap(frameWriter.toByteArray()), sigBob.build()); + } + +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java index 6ce7769ac91..3ba5dbb6b0f 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java @@ -27,7 +27,10 @@ import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.Map; @@ -79,6 +82,38 @@ public class MapOfColumnsRowsAndColumns implements RowsAndColumns ); } + public static MapOfColumnsRowsAndColumns fromRowObjects(Object[][] objs, RowSignature signature) + { + final Builder bob = builder(); + + if (objs.length > 0) { + Object[][] columnOriented = new Object[objs[0].length][objs.length]; + for (int i = 0; i < objs.length; ++i) { + for (int j = 0; j < objs[i].length; ++j) { + columnOriented[j][i] = objs[i][j]; + } + } + + for (int i = 0; i < signature.size(); ++i) { + final ColumnType type = signature.getColumnType(i).orElse(null); + + // If the column is String type, we likely got String objects instead of utf8 bytes, so convert to utf8Bytes + // to align with expectations. + if (ColumnType.STRING.equals(type)) { + for (int j = 0; j < columnOriented[i].length; j++) { + if (columnOriented[i][j] instanceof String) { + columnOriented[i][j] = ByteBuffer.wrap(((String) columnOriented[i][j]).getBytes(StandardCharsets.UTF_8)); + } + } + } + + bob.add(signature.getColumnName(i), columnOriented[i], type); + } + } + + return bob.build(); + } + private final Map mapOfColumns; private final int numRows; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectArrayColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectArrayColumn.java index 211c176902f..38467a83e39 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectArrayColumn.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectArrayColumn.java @@ -19,6 +19,7 @@ package org.apache.druid.query.rowsandcols.column; +import org.apache.druid.query.rowsandcols.column.accessor.ObjectColumnAccessorBase; import org.apache.druid.query.rowsandcols.util.FindResult; import org.apache.druid.segment.column.ColumnType; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/DoubleColumnAccessorBase.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/DoubleColumnAccessorBase.java new file mode 100644 index 00000000000..b1a04a95a05 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/DoubleColumnAccessorBase.java @@ -0,0 +1,74 @@ +/* + * 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.druid.query.rowsandcols.column.accessor; + +import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; + +public abstract class DoubleColumnAccessorBase implements ColumnAccessor +{ + @Override + public ColumnType getType() + { + return ColumnType.DOUBLE; + } + + @Nullable + @Override + public Object getObject(int rowNum) + { + return getDouble(rowNum); + } + + @Override + public float getFloat(int rowNum) + { + return (float) getDouble(rowNum); + } + + @Override + public long getLong(int rowNum) + { + return (long) getDouble(rowNum); + } + + @Override + public int getInt(int rowNum) + { + return (int) getDouble(rowNum); + } + + @Override + public int compareRows(int lhsRowNum, int rhsRowNum) + { + if (isNull(lhsRowNum)) { + return isNull(rhsRowNum) ? 0 : -1; + } + + if (isNull(rhsRowNum)) { + // if lhs was null, the code would've entered into the if clause before this one. + return 1; + } + + return Double.compare(getDouble(lhsRowNum), getDouble(rhsRowNum)); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/FloatColumnAccessorBase.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/FloatColumnAccessorBase.java new file mode 100644 index 00000000000..1c8892876cd --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/FloatColumnAccessorBase.java @@ -0,0 +1,74 @@ +/* + * 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.druid.query.rowsandcols.column.accessor; + +import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; + +public abstract class FloatColumnAccessorBase implements ColumnAccessor +{ + @Override + public ColumnType getType() + { + return ColumnType.FLOAT; + } + + @Nullable + @Override + public Object getObject(int rowNum) + { + return getFloat(rowNum); + } + + @Override + public double getDouble(int rowNum) + { + return getFloat(rowNum); + } + + @Override + public long getLong(int rowNum) + { + return (long) getFloat(rowNum); + } + + @Override + public int getInt(int rowNum) + { + return (int) getFloat(rowNum); + } + + @Override + public int compareRows(int lhsRowNum, int rhsRowNum) + { + if (isNull(lhsRowNum)) { + return isNull(rhsRowNum) ? 0 : -1; + } + + if (isNull(rhsRowNum)) { + // if lhs was null, the code would've entered into the if clause before this one. + return 1; + } + + return Float.compare(getFloat(lhsRowNum), getFloat(rhsRowNum)); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/LongColumnAccessorBase.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/LongColumnAccessorBase.java new file mode 100644 index 00000000000..595ab9eb4d6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/LongColumnAccessorBase.java @@ -0,0 +1,74 @@ +/* + * 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.druid.query.rowsandcols.column.accessor; + +import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; + +public abstract class LongColumnAccessorBase implements ColumnAccessor +{ + @Override + public ColumnType getType() + { + return ColumnType.LONG; + } + + @Nullable + @Override + public Object getObject(int rowNum) + { + return getLong(rowNum); + } + + @Override + public float getFloat(int rowNum) + { + return getLong(rowNum); + } + + @Override + public double getDouble(int rowNum) + { + return getLong(rowNum); + } + + @Override + public int getInt(int rowNum) + { + return (int) getLong(rowNum); + } + + @Override + public int compareRows(int lhsRowNum, int rhsRowNum) + { + if (isNull(lhsRowNum)) { + return isNull(rhsRowNum) ? 0 : -1; + } + + if (isNull(rhsRowNum)) { + // if lhs was null, the code would've entered into the if clause before this one. + return 1; + } + + return Long.compare(getLong(lhsRowNum), getLong(rhsRowNum)); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectColumnAccessorBase.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/ObjectColumnAccessorBase.java similarity index 95% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectColumnAccessorBase.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/ObjectColumnAccessorBase.java index a657058f25e..8286c6613d3 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectColumnAccessorBase.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/accessor/ObjectColumnAccessorBase.java @@ -17,7 +17,9 @@ * under the License. */ -package org.apache.druid.query.rowsandcols.column; +package org.apache.druid.query.rowsandcols.column.accessor; + +import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import javax.annotation.Nullable; import java.util.Comparator; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnHolderRACColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnHolderRACColumn.java new file mode 100644 index 00000000000..ed4f8ead52e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnHolderRACColumn.java @@ -0,0 +1,188 @@ +/* + * 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.druid.query.rowsandcols.concrete; + +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.column.BaseColumn; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.ReadableOffset; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.Comparator; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Implements a RowsAndColumns column based on a ColumnHolder. There is a bit of an impedence mis-match between + * these interfaces in that the ColumnHolder gets at the values of a row by using a BaseColumn which in turn + * hands off to `ColumnValueSelector` to actually build the thing. The ColumnAccessor from RowsAndColumns, however, + * believes that it can provide effectively direct access to values. + *

    + * It would be really good to eventually reach down and interact directly with the columns rather than doing this + * round-about stuff. + *

    + * This implementation is also very bad with Objects and data types, it masterfully avoids almost every optimization + * that we have in place. This should also be revisited at some point in time. + */ +public class ColumnHolderRACColumn implements Column, Closeable +{ + private final ColumnHolder holder; + + private BaseColumn baseColumn; + + public ColumnHolderRACColumn( + ColumnHolder holder + ) + { + this.holder = holder; + } + + @Nonnull + @Override + public ColumnAccessor toAccessor() + { + final BaseColumn baseColumn = getBaseColumn(); + + AtomicInteger offset = new AtomicInteger(0); + final ColumnValueSelector valueSelector = baseColumn.makeColumnValueSelector( + new AtomicIntegerReadableOffset(offset) + ); + final ColumnType columnType = holder.getCapabilities().toColumnType(); + final Comparator comparator = Comparator.nullsFirst(columnType.getStrategy()); + + return new ColumnAccessor() + { + @Override + public ColumnType getType() + { + return columnType; + } + + @Override + public int numRows() + { + return holder.getLength(); + } + + @Override + public boolean isNull(int rowNum) + { + offset.set(rowNum); + return valueSelector.isNull(); + } + + @Nullable + @Override + public Object getObject(int rowNum) + { + offset.set(rowNum); + return valueSelector.getObject(); + } + + @Override + public double getDouble(int rowNum) + { + offset.set(rowNum); + return valueSelector.getDouble(); + + } + + @Override + public float getFloat(int rowNum) + { + offset.set(rowNum); + return valueSelector.getFloat(); + + } + + @Override + public long getLong(int rowNum) + { + offset.set(rowNum); + return valueSelector.getLong(); + } + + @Override + public int getInt(int rowNum) + { + offset.set(rowNum); + return (int) valueSelector.getLong(); + + } + + @Override + public int compareRows(int lhsRowNum, int rhsRowNum) + { + return comparator.compare(getObject(lhsRowNum), getObject(rhsRowNum)); + } + }; + } + + @Nullable + @Override + public T as(Class clazz) + { + return null; + } + + @Override + public void close() throws IOException + { + if (baseColumn != null) { + baseColumn.close(); + } + } + + public BaseColumn getBaseColumn() + { + if (baseColumn == null) { + baseColumn = holder.getColumn(); + } + return baseColumn; + } + + private static class AtomicIntegerReadableOffset implements ReadableOffset + { + private final AtomicInteger offset; + + public AtomicIntegerReadableOffset(AtomicInteger offset) + { + this.offset = offset; + } + + @Override + public int getOffset() + { + return offset.get(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java new file mode 100644 index 00000000000..97237ef77a2 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java @@ -0,0 +1,86 @@ +/* + * 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.druid.query.rowsandcols.concrete; + +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.read.columnar.FrameColumnReaders; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.LinkedHashMap; + +public class FrameRowsAndColumns implements RowsAndColumns +{ + private final Frame frame; + private final RowSignature signature; + private final LinkedHashMap colCache = new LinkedHashMap<>(); + + public FrameRowsAndColumns(Frame frame, RowSignature signature) + { + this.frame = FrameType.COLUMNAR.ensureType(frame); + this.signature = signature; + } + + @Override + public Collection getColumnNames() + { + return signature.getColumnNames(); + } + + @Override + public int numRows() + { + return frame.numRows(); + } + + @Nullable + @Override + public Column findColumn(String name) + { + // Use contains so that we can negative cache. + if (!colCache.containsKey(name)) { + final int columnIndex = signature.indexOf(name); + if (columnIndex < 0) { + colCache.put(name, null); + } else { + final ColumnType columnType = signature + .getColumnType(columnIndex) + .orElseThrow(() -> new ISE("just got the id, why is columnType not there?")); + + colCache.put(name, FrameColumnReaders.create(columnIndex, columnType).readRACColumn(frame)); + } + } + return colCache.get(name); + + } + + @Nullable + @Override + public T as(Class clazz) + { + return null; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java new file mode 100644 index 00000000000..e2dc8fa0a9d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java @@ -0,0 +1,106 @@ +/* + * 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.druid.query.rowsandcols.concrete; + +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.segment.CloseableShapeshifter; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnHolder; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +public class QueryableIndexRowsAndColumns implements RowsAndColumns, AutoCloseable, CloseableShapeshifter +{ + private static final HashMap, Function> AS_MAP = makeAsMap(); + + private final QueryableIndex index; + + private final Closer closer = Closer.create(); + private final AtomicInteger numRows = new AtomicInteger(-1); + + public QueryableIndexRowsAndColumns( + QueryableIndex index + ) + { + this.index = index; + } + + @Override + public Collection getColumnNames() + { + return index.getColumnNames(); + } + + @Override + public int numRows() + { + int retVal = numRows.get(); + if (retVal < 0) { + retVal = index.getNumRows(); + numRows.set(retVal); + } + return retVal; + } + + @Nullable + @Override + public Column findColumn(String name) + { + final ColumnHolder columnHolder = index.getColumnHolder(name); + if (columnHolder == null) { + return null; + } + + return closer.register(new ColumnHolderRACColumn(columnHolder)); + } + + @SuppressWarnings("unchecked") + @Nullable + @Override + public T as(Class clazz) + { + return (T) AS_MAP.getOrDefault(clazz, arg -> null).apply(this); + } + + @Override + public void close() throws IOException + { + closer.close(); + } + + private static HashMap, Function> makeAsMap() + { + HashMap, Function> retVal = new HashMap<>(); + + retVal.put(StorageAdapter.class, rac -> new QueryableIndexStorageAdapter(rac.index)); + retVal.put(QueryableIndex.class, rac -> rac.index); + + return retVal; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java index ada5c18422e..a8f3570b9ad 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java @@ -19,7 +19,8 @@ package org.apache.druid.query.rowsandcols.semantic; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.rowsandcols.RowsAndColumns; @@ -32,11 +33,13 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.nio.ByteBuffer; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -80,15 +83,46 @@ public class DefaultColumnSelectorFactoryMaker implements ColumnSelectorFactoryM { return withColumnAccessor(dimensionSpec.getDimension(), columnAccessor -> { if (columnAccessor == null) { - return DimensionSelector.constant(null); + return DimensionSelector.nilSelector(); } else { + boolean maybeSupportsUtf8 = columnAccessor.getType().is(ValueType.STRING); + int rowCounter = 0; + int rowCount = columnAccessor.numRows(); + while (maybeSupportsUtf8 && rowCounter < rowCount && columnAccessor.isNull(rowCounter)) { + ++rowCounter; + } + + if (rowCounter == rowCount) { + // We iterated through all of the things and got only nulls, might as well specialize to a null + return DimensionSelector.nilSelector(); + } + + final boolean supportsUtf8 = maybeSupportsUtf8 && columnAccessor.getObject(rowCounter) instanceof ByteBuffer; + return new BaseSingleValueDimensionSelector() { @Nullable @Override protected String getValue() { - return String.valueOf(columnAccessor.getObject(cellIdSupplier.get())); + final Object retVal = columnAccessor.getObject(cellIdSupplier.get()); + if (retVal instanceof ByteBuffer) { + return StringUtils.fromUtf8(((ByteBuffer) retVal).asReadOnlyBuffer()); + } + return String.valueOf(retVal); + } + + @Nullable + @Override + public ByteBuffer lookupNameUtf8(int id) + { + return (ByteBuffer) columnAccessor.getObject(cellIdSupplier.get()); + } + + @Override + public boolean supportsLookupNameUtf8() + { + return supportsUtf8; } @Override @@ -109,91 +143,15 @@ public class DefaultColumnSelectorFactoryMaker implements ColumnSelectorFactoryM if (columnAccessor == null) { return DimensionSelector.constant(null); } else { - return new ColumnValueSelector() - { - private final AtomicReference myClazz = new AtomicReference<>(null); - - @Nullable - @Override - public Object getObject() - { - return columnAccessor.getObject(cellIdSupplier.get()); - } - - @SuppressWarnings("rawtypes") - @Override - public Class classOfObject() - { - Class retVal = myClazz.get(); - if (retVal == null) { - retVal = findClazz(); - myClazz.set(retVal); - } - return retVal; - } - - private Class findClazz() - { - final ColumnType type = columnAccessor.getType(); - switch (type.getType()) { - case LONG: - return long.class; - case DOUBLE: - return double.class; - case FLOAT: - return float.class; - case STRING: - return String.class; - case ARRAY: - return List.class; - case COMPLEX: - final ComplexMetricSerde serdeForType = ComplexMetrics.getSerdeForType(type.getComplexTypeName()); - if (serdeForType != null && serdeForType.getObjectStrategy() != null) { - return serdeForType.getObjectStrategy().getClazz(); - } - - for (int i = 0; i < columnAccessor.numRows(); ++i) { - Object obj = columnAccessor.getObject(i); - if (obj != null) { - return obj.getClass(); - } - } - return Object.class; - default: - throw new ISE("Unknown type[%s]", type.getType()); - } - } - - @Override - public boolean isNull() - { - return columnAccessor.isNull(cellIdSupplier.get()); - } - - @Override - public long getLong() - { - return columnAccessor.getLong(cellIdSupplier.get()); - } - - @Override - public float getFloat() - { - return columnAccessor.getFloat(cellIdSupplier.get()); - } - - @Override - public double getDouble() - { - return columnAccessor.getDouble(cellIdSupplier.get()); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - - } - }; + final ColumnType type = columnAccessor.getType(); + switch (type.getType()) { + case STRING: + return new StringColumnValueSelector(columnAccessor); + case COMPLEX: + return new ComplexColumnValueSelector(columnAccessor); + default: + return new PassThroughColumnValueSelector(columnAccessor); + } } }); } @@ -205,6 +163,7 @@ public class DefaultColumnSelectorFactoryMaker implements ColumnSelectorFactoryM return withColumnAccessor(column, columnAccessor -> new ColumnCapabilitiesImpl() .setType(columnAccessor.getType()) + .setHasMultipleValues(false) .setDictionaryEncoded(false) .setHasBitmapIndexes(false)); } @@ -219,5 +178,222 @@ public class DefaultColumnSelectorFactoryMaker implements ColumnSelectorFactoryM } return fn.apply(retVal); } + + private class PassThroughColumnValueSelector implements ColumnValueSelector + { + private final Class myClazz; + private final ColumnAccessor columnAccessor; + + public PassThroughColumnValueSelector( + ColumnAccessor columnAccessor + ) + { + this.columnAccessor = columnAccessor; + switch (columnAccessor.getType().getType()) { + case LONG: + myClazz = long.class; + break; + case DOUBLE: + myClazz = double.class; + break; + case FLOAT: + myClazz = float.class; + break; + case ARRAY: + myClazz = List.class; + default: + throw DruidException.defensive("this class cannot handle type [%s]", columnAccessor.getType()); + } + } + + @Nullable + @Override + public Object getObject() + { + return columnAccessor.getObject(cellIdSupplier.get()); + } + + @SuppressWarnings("rawtypes") + @Override + public Class classOfObject() + { + return myClazz; + } + + @Override + public boolean isNull() + { + return columnAccessor.isNull(cellIdSupplier.get()); + } + + @Override + public long getLong() + { + return columnAccessor.getLong(cellIdSupplier.get()); + } + + @Override + public float getFloat() + { + return columnAccessor.getFloat(cellIdSupplier.get()); + } + + @Override + public double getDouble() + { + return columnAccessor.getDouble(cellIdSupplier.get()); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + } + + private class StringColumnValueSelector implements ColumnValueSelector + { + private final ColumnAccessor columnAccessor; + + public StringColumnValueSelector( + ColumnAccessor columnAccessor + ) + { + this.columnAccessor = columnAccessor; + } + + @Nullable + @Override + public Object getObject() + { + // We want our String columns to be ByteBuffers, but users of this ColumnValueSelector interface + // would generally expect String objects instead of UTF8 ByteBuffers, so we have to convert here + // if we get a ByteBuffer. + + final Object retVal = columnAccessor.getObject(cellIdSupplier.get()); + if (retVal instanceof ByteBuffer) { + return StringUtils.fromUtf8(((ByteBuffer) retVal).asReadOnlyBuffer()); + } + return retVal; + } + + @SuppressWarnings("rawtypes") + @Override + public Class classOfObject() + { + return String.class; + } + + @Override + public boolean isNull() + { + return columnAccessor.isNull(cellIdSupplier.get()); + } + + @Override + public long getLong() + { + return columnAccessor.getLong(cellIdSupplier.get()); + } + + @Override + public float getFloat() + { + return columnAccessor.getFloat(cellIdSupplier.get()); + } + + @Override + public double getDouble() + { + return columnAccessor.getDouble(cellIdSupplier.get()); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + } + + private class ComplexColumnValueSelector implements ColumnValueSelector + { + private final AtomicReference myClazz; + private final ColumnAccessor columnAccessor; + + public ComplexColumnValueSelector(ColumnAccessor columnAccessor) + { + this.columnAccessor = columnAccessor; + myClazz = new AtomicReference<>(null); + } + + @Nullable + @Override + public Object getObject() + { + return columnAccessor.getObject(cellIdSupplier.get()); + } + + @SuppressWarnings("rawtypes") + @Override + public Class classOfObject() + { + Class retVal = myClazz.get(); + if (retVal == null) { + retVal = findClazz(); + myClazz.set(retVal); + } + return retVal; + } + + private Class findClazz() + { + final ColumnType type = columnAccessor.getType(); + if (type.getType() == ValueType.COMPLEX) { + final ComplexMetricSerde serdeForType = ComplexMetrics.getSerdeForType(type.getComplexTypeName()); + if (serdeForType != null && serdeForType.getObjectStrategy() != null) { + return serdeForType.getObjectStrategy().getClazz(); + } + + for (int i = 0; i < columnAccessor.numRows(); ++i) { + Object obj = columnAccessor.getObject(i); + if (obj != null) { + return obj.getClass(); + } + } + return Object.class; + } + throw DruidException.defensive("this class cannot handle type [%s]", type); + } + + @Override + public boolean isNull() + { + return columnAccessor.isNull(cellIdSupplier.get()); + } + + @Override + public long getLong() + { + return columnAccessor.getLong(cellIdSupplier.get()); + } + + @Override + public float getFloat() + { + return columnAccessor.getFloat(cellIdSupplier.get()); + } + + @Override + public double getDouble() + { + return columnAccessor.getDouble(cellIdSupplier.get()); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + } } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java new file mode 100644 index 00000000000..fd81491112d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java @@ -0,0 +1,149 @@ +/* + * 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.druid.query.rowsandcols.semantic; + +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.filter.AndFilter; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashSet; +import java.util.List; + +public class DefaultRowsAndColumnsDecorator implements RowsAndColumnsDecorator +{ + private RowsAndColumns base; + private Interval interval; + private Filter filter; + private VirtualColumns virtualColumns; + private int limit; + private List ordering; + + public DefaultRowsAndColumnsDecorator( + RowsAndColumns base + ) + { + this(base, null, null, null, -1, null); + } + + public DefaultRowsAndColumnsDecorator( + RowsAndColumns base, + Interval interval, + Filter filter, + VirtualColumns virtualColumns, + int limit, + List ordering + ) + { + this.base = base; + this.interval = interval; + this.filter = filter; + this.virtualColumns = virtualColumns; + this.limit = limit; + this.ordering = ordering; + } + + @Override + public void limitTimeRange(Interval interval) + { + if (this.interval == null) { + this.interval = interval; + } else { + this.interval = this.interval.overlap(interval); + } + } + + @Override + public void addFilter(Filter filter) + { + if (this.filter == null) { + this.filter = filter; + } else { + LinkedHashSet newFilters = new LinkedHashSet<>(); + if (this.filter instanceof AndFilter) { + newFilters.addAll(((AndFilter) this.filter).getFilters()); + } else { + newFilters.add(this.filter); + } + + newFilters.add(filter); + this.filter = new AndFilter(newFilters); + } + } + + @Override + public void addVirtualColumns(VirtualColumns virtualColumns) + { + if (this.virtualColumns == null) { + this.virtualColumns = virtualColumns; + } else { + final VirtualColumn[] existing = this.virtualColumns.getVirtualColumns(); + final VirtualColumn[] incoming = virtualColumns.getVirtualColumns(); + ArrayList cols = new ArrayList<>(existing.length + incoming.length); + cols.addAll(Arrays.asList(existing)); + cols.addAll(Arrays.asList(incoming)); + + this.virtualColumns = VirtualColumns.create(cols); + } + } + + @Override + public void setLimit(int numRows) + { + if (this.limit == -1) { + this.limit = numRows; + } else { + this.limit = Math.min(limit, numRows); + } + } + + @Override + public void setOrdering(List ordering) + { + this.ordering = ordering; + } + + @Override + public RowsAndColumns restrictColumns(List columns) + { + return new LazilyDecoratedRowsAndColumns( + base, + interval, + filter, + virtualColumns, + limit, + ordering, + columns == null ? null : new LinkedHashSet<>(columns) + ); + } + + @Override + public RowsAndColumns toRowsAndColumns() + { + return restrictColumns(null); + } + +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java new file mode 100644 index 00000000000..77d8e2068dd --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java @@ -0,0 +1,71 @@ +/* + * 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.druid.query.rowsandcols.semantic; + +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.segment.VirtualColumns; +import org.joda.time.Interval; + +import java.util.List; + +/** + * An interface for "decorating" a rowsAndColumns. This basically takes extra metadata that impacts the shape of the + * RowsAndColumns. + * + * Generally speaking, all of the void methods on this interface should cause the RowsAndColumns object to act + * as if it has been mutated with that decoration already. That is, whether an implementation is lazy or not should + * not impact what is visible/not visible. + * + * After all decoration methods have been called, either {@link #restrictColumns(List)} or {@link #toRowsAndColumns()} + * can be called to generate a RowsAndColumns with the decorations applied. Note, that it is generally expected that + * implementations will choose to lazily apply the decorations and not actually materialize them until the last + * possible moment, but this is an implementation detail left up to the specific implementation. + */ +public interface RowsAndColumnsDecorator +{ + static RowsAndColumnsDecorator fromRAC(RowsAndColumns rac) + { + if (rac instanceof RowsAndColumnsDecorator) { + return (RowsAndColumnsDecorator) rac; + } + + final RowsAndColumnsDecorator retVal = rac.as(RowsAndColumnsDecorator.class); + if (retVal == null) { + return new DefaultRowsAndColumnsDecorator(rac); + } + return retVal; + } + + void limitTimeRange(Interval interval); + + void addFilter(Filter filter); + + void addVirtualColumns(VirtualColumns virtualColumn); + + void setLimit(int numRows); + + void setOrdering(List ordering); + + RowsAndColumns restrictColumns(List columns); + + RowsAndColumns toRowsAndColumns(); +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/WireTransferable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/WireTransferable.java new file mode 100644 index 00000000000..a7d55f59929 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/WireTransferable.java @@ -0,0 +1,37 @@ +/* + * 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.druid.query.rowsandcols.semantic; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.rowsandcols.RowsAndColumns; + +public interface WireTransferable +{ + static WireTransferable fromRAC(RowsAndColumns rac) + { + WireTransferable retVal = rac.as(WireTransferable.class); + if (retVal == null) { + throw new ISE("Rac[%s] cannot be transferred over the wire", rac.getClass()); + } + return retVal; + } + + byte[] bytesToTransfer(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java index ba5b202cbae..12ab6c90197 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java @@ -148,10 +148,15 @@ public interface DimensionSelector extends ColumnValueSelector, Dimensio } } + static DimensionSelector nilSelector() + { + return NullDimensionSelectorHolder.NULL_DIMENSION_SELECTOR; + } + static DimensionSelector constant(@Nullable final String value) { if (NullHandling.isNullOrEquivalent(value)) { - return NullDimensionSelectorHolder.NULL_DIMENSION_SELECTOR; + return nilSelector(); } else { return new ConstantDimensionSelector(value); } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java index a829dfae184..f145c1db9cd 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java @@ -19,9 +19,13 @@ package org.apache.druid.segment; +import org.apache.druid.query.rowsandcols.concrete.QueryableIndexRowsAndColumns; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + /** */ public class QueryableIndexSegment implements Segment @@ -64,7 +68,19 @@ public class QueryableIndexSegment implements Segment @Override public void close() { - // this is kinda nasty + // this is kinda nasty because it actually unmaps the files and stuff too index.close(); } + + @SuppressWarnings("unchecked") + @Nullable + @Override + public T as(@Nonnull Class clazz) + { + if (CloseableShapeshifter.class.equals(clazz)) { + return (T) new QueryableIndexRowsAndColumns(index); + } + + return null; + } } diff --git a/processing/src/test/java/org/apache/druid/query/operator/OperatorSequenceTest.java b/processing/src/test/java/org/apache/druid/query/operator/OperatorSequenceTest.java index ffea250f6a0..a1b5e9f1aea 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/OperatorSequenceTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/OperatorSequenceTest.java @@ -140,13 +140,15 @@ public class OperatorSequenceTest yielder = yielder.next(0); Assert.assertTrue(yielder.isDone()); - // Aggregate each RAC and yield. + // Aggregate each RAC and yield every other. yielder = seq.toYielder(0, new YieldingAccumulator() { @Override public Integer accumulate(Integer accumulated, RowsAndColumns in) { - this.yield(); + if (accumulated != 0) { + this.yield(); + } final ColumnAccessor col = in.findColumn("hi").toAccessor(); for (int i = 0; i < col.numRows(); ++i) { accumulated += col.getInt(i); @@ -155,7 +157,7 @@ public class OperatorSequenceTest } }); - int[] expectedTotals = new int[]{1, 2, 7, 26, 30, 54}; + int[] expectedTotals = new int[]{3, 33, 84}; for (int expectedTotal : expectedTotals) { Assert.assertEquals(expectedTotal, yielder.get().intValue()); diff --git a/processing/src/test/java/org/apache/druid/query/operator/OperatorTestHelper.java b/processing/src/test/java/org/apache/druid/query/operator/OperatorTestHelper.java index e44bdc58067..7900dcdad3b 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/OperatorTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/operator/OperatorTestHelper.java @@ -20,12 +20,14 @@ package org.apache.druid.query.operator; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.junit.Assert; +import java.io.Closeable; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Supplier; @@ -37,7 +39,7 @@ public class OperatorTestHelper public OperatorTestHelper expectRowsAndColumns(RowsAndColumnsHelper... helpers) { return withPushFn( - new JustPushMe() + () -> new JustPushMe() { int index = 0; @@ -56,7 +58,7 @@ public class OperatorTestHelper public OperatorTestHelper expectAndStopAfter(RowsAndColumnsHelper... helpers) { return withPushFn( - new JustPushMe() + () -> new JustPushMe() { int index = 0; @@ -95,12 +97,12 @@ public class OperatorTestHelper return this; } - public OperatorTestHelper withPushFn(JustPushMe fn) + public OperatorTestHelper withPushFn(Supplier fnSupplier) { - return withReceiver(() -> new TestReceiver(fn)); + return withReceiver(() -> new TestReceiver(fnSupplier.get())); } - public OperatorTestHelper runToCompletion(Operator op) + public void runToCompletion(Operator op) { TestReceiver receiver = this.receiverSupply.get(); Operator.go(op, receiver); @@ -108,7 +110,36 @@ public class OperatorTestHelper if (finalValidation != null) { finalValidation.accept(receiver); } - return this; + + for (int i = 1; i < receiver.getNumPushed(); ++i) { + long expectedNumPauses = receiver.getNumPushed() / i; + if (receiver.getNumPushed() % i > 0) { + ++expectedNumPauses; + } + runWhilePausing(op, expectedNumPauses, i); + } + } + + private void runWhilePausing(Operator op, long expectedNumPauses, int pauseAfter) + { + // We are now going to do the same pushes and the same validation, but pausing after every possible point + // that we could pause. It should still produce the same results as running through without any pauses. + TestReceiver pausingReceiver = this.receiverSupply.get(); + pausingReceiver.setPauseAfter(pauseAfter); + + int numPauses = 0; + Closeable continuation = null; + do { + continuation = op.goOrContinue(continuation, pausingReceiver); + ++numPauses; + } while (continuation != null); + + final String msg = StringUtils.format("pauseAfter[%,d]", pauseAfter); + Assert.assertTrue(msg, pausingReceiver.isCompleted()); + Assert.assertEquals(msg, expectedNumPauses, numPauses); + if (finalValidation != null) { + finalValidation.accept(pausingReceiver); + } } public interface JustPushMe @@ -120,8 +151,9 @@ public class OperatorTestHelper { private final JustPushMe pushFn; - private AtomicLong numPushed = new AtomicLong(); + private AtomicInteger numPushed = new AtomicInteger(); private AtomicBoolean completed = new AtomicBoolean(false); + private long pauseAfter = -1; public TestReceiver(JustPushMe pushFn) { @@ -132,7 +164,14 @@ public class OperatorTestHelper public Operator.Signal push(RowsAndColumns rac) { numPushed.incrementAndGet(); - return pushFn.push(rac); + + final Operator.Signal push = pushFn.push(rac); + + if (push == Operator.Signal.GO && pauseAfter != -1 && numPushed.get() % pauseAfter == 0) { + return Operator.Signal.PAUSE; + } + + return push; } public boolean isCompleted() @@ -148,9 +187,14 @@ public class OperatorTestHelper } } - public long getNumPushed() + public int getNumPushed() { return numPushed.get(); } + + public void setPauseAfter(long pauseAfter) + { + this.pauseAfter = pauseAfter; + } } } diff --git a/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java new file mode 100644 index 00000000000..2b02d161d7a --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java @@ -0,0 +1,296 @@ +/* + * 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.druid.query.operator; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.DimFilters; +import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.TestRowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.RowsAndColumnsDecorator; +import org.apache.druid.query.rowsandcols.semantic.TestRowsAndColumnsDecorator; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +@SuppressWarnings("unchecked") +public class ScanOperatorFactoryTest +{ + static { + NullHandling.initializeForTests(); + } + + @Test + public void testEquals() + { + final Builder bob = new Builder(); + bob.timeRange = Intervals.utc(0, 6); + bob.filter = DimFilters.dimEquals("abc", "b"); + bob.limit = 48; + bob.projectedColumns = Arrays.asList("a", "b"); + bob.virtualColumns = VirtualColumns.EMPTY; + bob.ordering = Collections.singletonList(ColumnWithDirection.ascending("a")); + ScanOperatorFactory factory = bob.build(); + + Assert.assertEquals(factory, factory); + Assert.assertNotEquals(factory, new Object()); + + Assert.assertNotEquals(factory, bob.copy().setTimeRange(null).build()); + Assert.assertNotEquals(factory, bob.copy().setFilter(null).build()); + Assert.assertNotEquals(factory, bob.copy().setLimit(null).build()); + Assert.assertNotEquals(factory, bob.copy().setProjectedColumns(null).build()); + Assert.assertNotEquals(factory, bob.copy().setVirtualColumns(null).build()); + Assert.assertNotEquals(factory, bob.copy().setOrdering(null).build()); + } + + @Test + public void testWrappedOperatorCarriesThroughValues() throws JsonProcessingException + { + ObjectMapper mapper = new DefaultObjectMapper(); + mapper.setInjectableValues(new InjectableValues.Std().addValue(ExprMacroTable.class, TestExprMacroTable.INSTANCE)); + + Interval[] intervals = new Interval[]{Intervals.utc(0, 6), Intervals.utc(6, 13), Intervals.utc(4, 8)}; + DimFilter[] filters = new DimFilter[]{ + new InDimFilter("dim", ImmutableSet.of("a", "b", "c", "e", "g")), + DimFilters.and( + new InDimFilter("dim", ImmutableSet.of("a", "b", "g")), + DimFilters.dimEquals("val", "789") + ), + DimFilters.or( + DimFilters.dimEquals("dim", "b"), + DimFilters.dimEquals("val", "789") + ), + DimFilters.dimEquals("dim", "f") + }; + int[] limits = new int[]{100, 1000}; + List[] orderings = new List[]{ + Arrays.asList(ColumnWithDirection.descending("__time"), ColumnWithDirection.ascending("dim")), + Collections.singletonList(ColumnWithDirection.ascending("val")), + Collections.emptyList() + }; + List[] projections = new List[]{ + Arrays.asList("dim", "val"), + Collections.singletonList("dim"), + Collections.emptyList() + }; + VirtualColumns[] virtualCols = new VirtualColumns[]{ + VirtualColumns.create(Collections.singletonList( + new ExpressionVirtualColumn("test", "2", null, TestExprMacroTable.INSTANCE) + )) + }; + + for (int i = 0; i <= intervals.length; ++i) { + Interval interval = (i == 0 ? null : intervals[i - 1]); + for (int j = 0; j <= filters.length; ++j) { + DimFilter filter = (j == 0 ? null : filters[j - 1]); + for (int k = 0; k <= limits.length; ++k) { + int limit = (k == 0 ? -1 : limits[k - 1]); + for (int l = 0; l <= orderings.length; ++l) { + List ordering = (l == 0) ? null : orderings[l - 1]; + for (int m = 0; m <= projections.length; ++m) { + List projection = (m == 0) ? null : projections[m - 1]; + for (int n = 0; n <= virtualCols.length; ++n) { + VirtualColumns virtual = (n == 0) ? VirtualColumns.EMPTY : virtualCols[n - 1]; + + + String msg = StringUtils.format( + "interval[%s], filter[%s], limit[%s], ordering[%s], projection[%s], virtual[%s]", + interval, + filter, + limit, + ordering, + projection, + virtual + ); + + ScanOperatorFactory factory = new ScanOperatorFactory( + interval, + filter, + limit, + projection, + virtual, + ordering + ); + + final String asString = mapper.writeValueAsString(factory); + final ScanOperatorFactory deserialized = mapper.readValue(asString, ScanOperatorFactory.class); + + Assert.assertEquals(msg, factory, deserialized); + Assert.assertEquals(msg, factory.hashCode(), deserialized.hashCode()); + + final ScanOperator wrapped = (ScanOperator) factory.wrap(new Operator() + { + @Nullable + @Override + public Closeable goOrContinue( + Closeable continuationObject, + Receiver receiver + ) + { + receiver.push(new TestRowsAndColumns().withAsImpl( + RowsAndColumnsDecorator.class, + TestRowsAndColumnsDecorator::new + )); + receiver.completed(); + return null; + } + }); + + Operator.go( + wrapped, + new Operator.Receiver() + { + @Override + public Operator.Signal push(RowsAndColumns inRac) + { + TestRowsAndColumnsDecorator.DecoratedRowsAndColumns rac = + (TestRowsAndColumnsDecorator.DecoratedRowsAndColumns) inRac; + + Assert.assertEquals(msg, factory.getTimeRange(), rac.getTimeRange()); + Assert.assertEquals(msg, factory.getLimit(), rac.getLimit()); + Assert.assertEquals(msg, factory.getVirtualColumns(), rac.getVirtualColumns()); + validateList(msg, factory.getOrdering(), rac.getOrdering()); + validateList(msg, factory.getProjectedColumns(), rac.getProjectedColumns()); + + Assert.assertEquals( + msg, + factory.getFilter() == null ? null : factory.getFilter().toFilter(), + rac.getFilter() + ); + + return Operator.Signal.GO; + } + + @Override + public void completed() + { + + } + } + ); + } + } + } + } + } + } + } + + private static void validateList( + String msg, + List expectedList, + List actualList + ) + { + if (expectedList != null && expectedList.isEmpty()) { + Assert.assertNull(msg, actualList); + } else { + Assert.assertEquals(msg, expectedList, actualList); + } + } + + private static class Builder + { + private Interval timeRange; + private DimFilter filter; + private Integer limit; + private List projectedColumns; + private VirtualColumns virtualColumns; + private List ordering; + + public Builder setTimeRange(Interval timeRange) + { + this.timeRange = timeRange; + return this; + } + + public Builder setFilter(DimFilter filter) + { + this.filter = filter; + return this; + } + + public Builder setLimit(Integer limit) + { + this.limit = limit; + return this; + } + + public Builder setProjectedColumns(List projectedColumns) + { + this.projectedColumns = projectedColumns; + return this; + } + + public Builder setVirtualColumns(VirtualColumns virtualColumns) + { + this.virtualColumns = virtualColumns; + return this; + } + + public Builder setOrdering(List ordering) + { + this.ordering = ordering; + return this; + } + + private Builder copy() + { + Builder retVal = new Builder(); + retVal.timeRange = timeRange; + retVal.filter = filter; + retVal.limit = limit; + retVal.projectedColumns = projectedColumns; + retVal.virtualColumns = virtualColumns; + retVal.ordering = ordering; + return retVal; + } + + private ScanOperatorFactory build() + { + return new ScanOperatorFactory( + timeRange, + filter, + limit, + projectedColumns, + virtualColumns, + ordering + ); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/SegmentToRowsAndColumnsOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/SegmentToRowsAndColumnsOperatorTest.java index f943f31201d..e89a0721691 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/SegmentToRowsAndColumnsOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/SegmentToRowsAndColumnsOperatorTest.java @@ -169,7 +169,7 @@ public class SegmentToRowsAndColumnsOperatorTest boolean exceptionThrown = false; try { new OperatorTestHelper() - .withPushFn(rac -> { + .withPushFn(() -> rac -> { Assert.assertSame(expectedRac, rac); return Operator.Signal.GO; }) diff --git a/processing/src/test/java/org/apache/druid/query/operator/SequenceOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/SequenceOperatorTest.java index b5bcc177aab..95f5bd00bc3 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/SequenceOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/SequenceOperatorTest.java @@ -43,12 +43,7 @@ public class SequenceOperatorTest .allColumnsRegistered(); new OperatorTestHelper() - .withPushFn( - rac -> { - helper.validate(rac); - return Operator.Signal.GO; - } - ) + .expectRowsAndColumns(helper, helper) .withFinalValidation(testReceiver -> Assert.assertEquals(2, testReceiver.getNumPushed())) .runToCompletion(op); } diff --git a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java index 81b684fdc33..69712c06493 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java @@ -20,10 +20,11 @@ package org.apache.druid.query.operator; import com.google.common.collect.ImmutableMap; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.QueryContext; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.segment.column.RowSignature; import org.junit.Assert; import org.junit.Before; @@ -50,9 +51,11 @@ public class WindowOperatorQueryTest { query = new WindowOperatorQuery( InlineDataSource.fromIterable(new ArrayList<>(), RowSignature.empty()), + new LegacySegmentSpec(Intervals.ETERNITY), ImmutableMap.of("sally", "sue"), RowSignature.empty(), - new ArrayList<>() + new ArrayList<>(), + null ); } @@ -100,15 +103,8 @@ public class WindowOperatorQueryTest final Set tableNames = query.getDataSource().getTableNames(); Assert.assertEquals(0, tableNames.size()); - boolean exceptionThrown = false; - try { - query.withDataSource(new TableDataSource("bob")); - } - catch (IAE e) { - // should fail trying to set a TableDataSource as TableDataSource is not currently allowed. - exceptionThrown = true; - } - Assert.assertTrue(exceptionThrown); + final TableDataSource newDs = new TableDataSource("bob"); + Assert.assertSame(newDs, query.withDataSource(newDs).getDataSource()); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java index b3a47164322..9cce74cb98c 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java @@ -58,7 +58,7 @@ public class WindowProcessorOperatorTest ); new OperatorTestHelper() - .withPushFn( + .withPushFn(() -> rowsAndColumns -> { Assert.assertSame(rac, rowsAndColumns); return Operator.Signal.GO; diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/TestRowsAndColumns.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/TestRowsAndColumns.java new file mode 100644 index 00000000000..40b70e855e4 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/TestRowsAndColumns.java @@ -0,0 +1,73 @@ +/* + * 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.druid.query.rowsandcols; + +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.query.rowsandcols.column.Column; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.function.Supplier; + +public class TestRowsAndColumns implements RowsAndColumns +{ + @SuppressWarnings("rawtypes") + private final Map asExpectations = new LinkedHashMap<>(); + + @Override + public Collection getColumnNames() + { + throw new UnsupportedOperationException(); + } + + @Override + public int numRows() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Column findColumn(String name) + { + throw new UnsupportedOperationException(); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + @Nullable + @Override + public T as(Class clazz) + { + final Supplier supplier = asExpectations.get(clazz); + if (supplier == null) { + throw new UOE("Cannot become class [%s], maybe need to call withAsImpl?", clazz); + } else { + return (T) supplier.get(); + } + } + + public TestRowsAndColumns withAsImpl(Class clazz, Supplier supp) + { + asExpectations.put(clazz, supp); + return this; + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java new file mode 100644 index 00000000000..3426dd00946 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.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.druid.query.rowsandcols.semantic; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.segment.ArrayListSegment; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.TypeStrategy; +import org.apache.druid.segment.filter.AndFilter; +import org.apache.druid.segment.filter.OrFilter; +import org.apache.druid.segment.filter.SelectorFilter; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.function.Function; + +@SuppressWarnings({"unchecked", "rawtypes", "ConstantConditions", "SingleStatementInBlock", "VariableNotUsedInsideIf"}) +public class RowsAndColumnsDecoratorTest extends SemanticTestBase +{ + public RowsAndColumnsDecoratorTest( + String name, + Function fn + ) + { + super(name, fn); + } + + @Test + public void testDecoration() + { + Object[][] vals = new Object[][]{ + {1L, "a", 123L, 0L}, + {2L, "a", 456L, 1L}, + {3L, "b", 789L, 2L}, + {4L, "b", 123L, 3L}, + {5L, "c", 456L, 4L}, + {6L, "c", 789L, 5L}, + {7L, "c", 123L, 6L}, + {8L, "d", 456L, 7L}, + {9L, "e", 789L, 8L}, + {10L, "f", 123L, 9L}, + {11L, "f", 456L, 10L}, + {12L, "g", 789L, 11L}, + }; + + RowSignature siggy = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim", ColumnType.STRING) + .add("val", ColumnType.LONG) + .add("arrayIndex", ColumnType.LONG) + .build(); + + final RowsAndColumns base = make(MapOfColumnsRowsAndColumns.fromRowObjects(vals, siggy)); + + Interval[] intervals = new Interval[]{Intervals.utc(0, 6), Intervals.utc(6, 13), Intervals.utc(4, 8)}; + Filter[] filters = new Filter[]{ + new InDimFilter("dim", ImmutableSet.of("a", "b", "c", "e", "g")), + new AndFilter(Arrays.asList( + new InDimFilter("dim", ImmutableSet.of("a", "b", "g")), + new SelectorFilter("val", "789") + )), + new OrFilter(Arrays.asList( + new SelectorFilter("dim", "b"), + new SelectorFilter("val", "789") + )), + new SelectorFilter("dim", "f") + }; + int[] limits = new int[]{3, 6, 100}; + List[] orderings = new List[]{ + Arrays.asList(ColumnWithDirection.descending("__time"), ColumnWithDirection.ascending("dim")), + Collections.singletonList(ColumnWithDirection.ascending("val")) + }; + + // call the same method multiple times + + for (int i = 0; i <= intervals.length; ++i) { + Interval interval = (i == 0 ? null : intervals[i - 1]); + for (int j = 0; j <= filters.length; ++j) { + Filter filter = (j == 0 ? null : filters[j - 1]); + for (int k = 0; k <= limits.length; ++k) { + int limit = (k == 0 ? -1 : limits[k - 1]); + for (int l = 0; l <= orderings.length; ++l) { + validateDecorated(base, siggy, vals, interval, filter, limit, l == 0 ? null : orderings[l - 1]); + } + } + } + } + } + + private void validateDecorated( + RowsAndColumns base, + RowSignature siggy, + Object[][] originalVals, + Interval interval, + Filter filter, + int limit, + List ordering + ) + { + String msg = StringUtils.format( + "interval[%s], filter[%s], limit[%s], ordering[%s]", + interval, + filter, + limit, + ordering + ); + RowsAndColumnsDecorator decor = RowsAndColumnsDecorator.fromRAC(base); + List vals; + + if (interval == null && filter == null) { + vals = Arrays.asList(originalVals); + } else { + decor.limitTimeRange(interval); + decor.addFilter(filter); + + final ArrayListSegment seggy = new ArrayListSegment<>( + SegmentId.dummy("dummy"), + new ArrayList<>(Arrays.asList(originalVals)), + columnName -> { + int index = siggy.indexOf(columnName); + return arr -> arr[index]; + }, + siggy + ); + final Sequence cursors = seggy + .asStorageAdapter() + .makeCursors( + filter, + interval == null ? Intervals.ETERNITY : interval, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ); + + vals = cursors.accumulate( + new ArrayList<>(), + (accumulated, in) -> { + final ColumnValueSelector idSupplier = in.getColumnSelectorFactory().makeColumnValueSelector("arrayIndex"); + while (!in.isDone()) { + accumulated.add(originalVals[(int) idSupplier.getLong()]); + in.advance(); + } + return accumulated; + } + ); + } + + if (ordering != null) { + decor.setOrdering(ordering); + + Comparator comparator = null; + for (ColumnWithDirection order : ordering) { + final int columnNum = siggy.indexOf(order.getColumn()); + final TypeStrategy strategy = + siggy.getColumnType(columnNum).orElseThrow(() -> new UOE("debug me")).getStrategy(); + + final Comparator newComp = (lhs, rhs) -> + strategy.compare(lhs[columnNum], rhs[columnNum]) * order.getDirection().getDirectionInt(); + + if (comparator == null) { + comparator = newComp; + } else { + comparator = comparator.thenComparing(newComp); + } + } + + vals = new ArrayList<>(vals); + + vals.sort(comparator); + } + + if (limit != -1) { + decor.setLimit(limit); + + vals = vals.subList(0, Math.min(vals.size(), limit)); + } + + if (ordering != null) { + Assert.assertThrows(msg, ISE.class, () -> decor.toRowsAndColumns().numRows()); + } else { + final RowsAndColumns rac = decor.toRowsAndColumns(); + Assert.assertEquals(msg, vals.size(), rac.numRows()); + + ColumnAccessor[] accessors = new ColumnAccessor[siggy.size()]; + for (int i = 0; i < siggy.size(); ++i) { + accessors[i] = rac.findColumn(siggy.getColumnName(i)).toAccessor(); + } + + for (int i = 0; i < vals.size(); ++i) { + Object[] actuals = new Object[accessors.length]; + for (int j = 0; j < actuals.length; ++j) { + actuals[j] = accessors[j].getObject(i); + if (actuals[j] instanceof ByteBuffer) { + actuals[j] = StringUtils.fromUtf8(((ByteBuffer) actuals[j]).asReadOnlyBuffer()); + } + } + Assert.assertArrayEquals(StringUtils.format("%s, row[%,d]", msg, i), vals.get(i), actuals); + } + } + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java new file mode 100644 index 00000000000..ffddbb3f743 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java @@ -0,0 +1,144 @@ +/* + * 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.druid.query.rowsandcols.semantic; + +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.segment.VirtualColumns; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.List; + +public class TestRowsAndColumnsDecorator implements RowsAndColumnsDecorator +{ + private Interval timeRange; + private Filter filter; + private VirtualColumns virtualColumns; + private int limit = -1; + private List ordering; + private List projectedColumns; + + @Override + public void limitTimeRange(Interval interval) + { + this.timeRange = interval; + } + + @Override + public void addFilter(Filter filter) + { + this.filter = filter; + } + + @Override + public void addVirtualColumns(VirtualColumns virtualColumns) + { + this.virtualColumns = virtualColumns; + } + + @Override + public void setLimit(int numRows) + { + this.limit = numRows; + } + + @Override + public void setOrdering(List ordering) + { + this.ordering = ordering; + } + + @Override + public RowsAndColumns restrictColumns(List columns) + { + this.projectedColumns = columns; + return toRowsAndColumns(); + } + + @Override + public RowsAndColumns toRowsAndColumns() + { + return new DecoratedRowsAndColumns(); + } + + public class DecoratedRowsAndColumns implements RowsAndColumns + { + public Interval getTimeRange() + { + return timeRange; + } + + public Filter getFilter() + { + return filter; + } + + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + + public int getLimit() + { + return limit; + } + + public List getOrdering() + { + return ordering; + } + + public List getProjectedColumns() + { + return projectedColumns; + } + + + @Override + public Collection getColumnNames() + { + throw new UnsupportedOperationException(); + } + + @Override + public int numRows() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Column findColumn(String name) + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public T as(Class clazz) + { + throw new UnsupportedOperationException(); + } + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 11f7db35ac5..8e0035c7e9c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -43,6 +43,7 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.util.ImmutableBitSet; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -50,6 +51,7 @@ import org.apache.druid.query.DataSource; import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongMinAggregatorFactory; @@ -64,6 +66,7 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.query.ordering.StringComparator; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.query.timeboundary.TimeBoundaryQuery; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.topn.DimensionTopNMetricSpec; @@ -815,7 +818,7 @@ public class DruidQuery /** * Whether the given datasource can make use of "intervals" based filtering. The is true for anything based on - * regular tables ({@link org.apache.druid.query.TableDataSource}). + * regular tables ({@link TableDataSource}). */ private static boolean canUseIntervalFiltering(final DataSource dataSource) { @@ -1380,11 +1383,21 @@ public class DruidQuery return null; } + final DataSource myDataSource; + if (dataSource instanceof TableDataSource) { + // In this case, we first plan a scan query to pull the results up for us before applying the window + myDataSource = new QueryDataSource(toScanQuery()); + } else { + myDataSource = dataSource; + } + return new WindowOperatorQuery( - dataSource, + myDataSource, + new LegacySegmentSpec(Intervals.ETERNITY), plannerContext.queryContextMap(), windowing.getSignature(), - windowing.getOperators() + windowing.getOperators(), + null ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index af5983d6edb..e486a958812 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -28,7 +28,6 @@ import org.apache.calcite.rel.RelFieldCollation; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.core.Window; -import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; @@ -88,25 +87,25 @@ public class Windowing { private static final ImmutableMap KNOWN_WINDOW_FNS = ImmutableMap .builder() - .put("LAG", (agg, typeFactory) -> - new WindowOffsetProcessor(agg.getColumn(typeFactory, 0), agg.getOutputName(), -agg.getConstantInt(1))) - .put("LEAD", (agg, typeFactory) -> - new WindowOffsetProcessor(agg.getColumn(typeFactory, 0), agg.getOutputName(), agg.getConstantInt(1))) - .put("FIRST_VALUE", (agg, typeFactory) -> - new WindowFirstProcessor(agg.getColumn(typeFactory, 0), agg.getOutputName())) - .put("LAST_VALUE", (agg, typeFactory) -> - new WindowLastProcessor(agg.getColumn(typeFactory, 0), agg.getOutputName())) - .put("CUME_DIST", (agg, typeFactory) -> + .put("LAG", (agg) -> + new WindowOffsetProcessor(agg.getColumn(0), agg.getOutputName(), -agg.getConstantInt(1))) + .put("LEAD", (agg) -> + new WindowOffsetProcessor(agg.getColumn(0), agg.getOutputName(), agg.getConstantInt(1))) + .put("FIRST_VALUE", (agg) -> + new WindowFirstProcessor(agg.getColumn(0), agg.getOutputName())) + .put("LAST_VALUE", (agg) -> + new WindowLastProcessor(agg.getColumn(0), agg.getOutputName())) + .put("CUME_DIST", (agg) -> new WindowCumeDistProcessor(agg.getGroup().getOrderingColumNames(), agg.getOutputName())) - .put("DENSE_RANK", (agg, typeFactory) -> + .put("DENSE_RANK", (agg) -> new WindowDenseRankProcessor(agg.getGroup().getOrderingColumNames(), agg.getOutputName())) - .put("NTILE", (agg, typeFactory) -> + .put("NTILE", (agg) -> new WindowPercentileProcessor(agg.getOutputName(), agg.getConstantInt(0))) - .put("PERCENT_RANK", (agg, typeFactory) -> + .put("PERCENT_RANK", (agg) -> new WindowRankProcessor(agg.getGroup().getOrderingColumNames(), agg.getOutputName(), true)) - .put("RANK", (agg, typeFactory) -> + .put("RANK", (agg) -> new WindowRankProcessor(agg.getGroup().getOrderingColumNames(), agg.getOutputName(), false)) - .put("ROW_NUMBER", (agg, typeFactory) -> + .put("ROW_NUMBER", (agg) -> new WindowRowNumberProcessor(agg.getOutputName())) .build(); private final List ops; @@ -196,18 +195,20 @@ public class Windowing aggregations.add(Iterables.getOnlyElement(aggregation.getAggregatorFactories())); } else { - processors.add(maker.make( - new WindowAggregate( - aggName, - aggregateCall, - sourceRowSignature, - plannerContext, - partialQuery.getSelectProject(), - window.constants, - group - ), - rexBuilder.getTypeFactory() - )); + processors.add( + maker.make( + new WindowAggregate( + aggName, + aggregateCall, + sourceRowSignature, + plannerContext, + rexBuilder, + partialQuery.getSelectProject(), + window.constants, + group + ) + ) + ); } } @@ -280,7 +281,7 @@ public class Windowing private interface ProcessorMaker { - Processor make(WindowAggregate agg, RelDataTypeFactory typeFactory); + Processor make(WindowAggregate agg); } private static class WindowGroup @@ -377,6 +378,7 @@ public class Windowing private final AggregateCall call; private final RowSignature sig; private final PlannerContext context; + private final RexBuilder rexBuilder; private final Project project; private final List constants; private final WindowGroup group; @@ -386,6 +388,7 @@ public class Windowing AggregateCall call, RowSignature sig, PlannerContext context, + RexBuilder rexBuilder, Project project, List constants, WindowGroup group @@ -395,6 +398,7 @@ public class Windowing this.call = call; this.sig = sig; this.context = context; + this.rexBuilder = rexBuilder; this.project = project; this.constants = constants; this.group = group; @@ -414,10 +418,10 @@ public class Windowing return group; } - public String getColumn(final RelDataTypeFactory typeFactory, int argPosition) + public String getColumn(int argPosition) { final RexNode columnArgument = - Expressions.fromFieldAccess(typeFactory, sig, project, call.getArgList().get(argPosition)); + Expressions.fromFieldAccess(rexBuilder.getTypeFactory(), sig, project, call.getArgList().get(argPosition)); final DruidExpression expression = Expressions.toDruidExpression(context, sig, columnArgument); if (expression == null) { throw new ISE("Couldn't get an expression from columnArgument[%s]", columnArgument); @@ -434,6 +438,14 @@ public class Windowing { return ((Number) getConstantArgument(argPosition).getValue()).intValue(); } + + public int getConstantInt(int argPosition, int defaultValue) + { + if (argPosition >= call.getArgList().size()) { + return defaultValue; + } + return getConstantInt(argPosition); + } } /** @@ -489,10 +501,9 @@ public class Windowing ) { final Iterator priorIterator = priorSort.iterator(); - final Iterator currentIterator = currentSort.iterator(); - while (currentIterator.hasNext()) { - if (!priorIterator.hasNext() || !currentIterator.next().equals(priorIterator.next())) { + for (ColumnWithDirection columnWithDirection : currentSort) { + if (!priorIterator.hasNext() || !columnWithDirection.equals(priorIterator.next())) { return false; } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java index 7fba1c581fd..48bf6ac1d73 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java @@ -102,7 +102,7 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest final URL systemResource = ClassLoader.getSystemResource("calcite/tests/window/" + filename); - final Object objectFromYaml = YAML_JACKSON.readValue(systemResource.openStream(), Object.class); + final Object objectFromYaml = YAML_JACKSON.readValue(systemResource, Object.class); final ObjectMapper queryJackson = queryFramework().queryJsonMapper(); final WindowQueryTestInputClass input = queryJackson.convertValue(objectFromYaml, WindowQueryTestInputClass.class); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java new file mode 100644 index 00000000000..84f920878f1 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.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.druid.sql.calcite; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterators; +import com.google.common.io.ByteStreams; +import com.google.inject.Injector; +import org.apache.commons.io.FileUtils; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.DoubleDimensionSchema; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nonnull; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Map; + +/** + * These test cases are borrowed from the drill-test-framework at https://github.com/apache/drill-test-framework + *

    + * The Drill data sources are just accessing parquet files directly, we ingest and index the data first via JSON + * (so that we avoid pulling in the parquet dependencies here) and then run the queries over that. + *

    + * The setup of the ingestion is done via code in this class, so any new data source needs to be added through that + * manner. That said, these tests are primarily being added to bootstrap our own test coverage of window + * functions, so it is believed that most iteration on tests will happen through the CalciteWindowQueryTest + * instead of this class. + */ +@RunWith(Parameterized.class) +public class DrillWindowQueryTest extends BaseCalciteQueryTest +{ + private static final Logger log = new Logger(DrillWindowQueryTest.class); + + static { + NullHandling.initializeForTests(); + } + + @Parameterized.Parameters(name = "{0}") + public static Object parametersForWindowQueryTest() throws Exception + { + final URL windowQueriesUrl = ClassLoader.getSystemResource("drill/window/queries/"); + File windowFolder = new File(windowQueriesUrl.toURI()); + int windowFolderPrefixSize = windowFolder.getAbsolutePath().length() + 1 /* 1 for the ending slash */; + + return FileUtils + .streamFiles(windowFolder, true, "q") + .map(file -> { + final String filePath = file.getAbsolutePath(); + return filePath.substring(windowFolderPrefixSize, filePath.length() - 2); + }) + .sorted() + .toArray(); + } + + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + private final String filename; + + public DrillWindowQueryTest( + String filename + ) + { + this.filename = filename; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker( + QueryRunnerFactoryConglomerate conglomerate, + JoinableFactoryWrapper joinableFactory, + Injector injector + ) throws IOException + { + final SpecificSegmentsQuerySegmentWalker retVal = super.createQuerySegmentWalker( + conglomerate, + joinableFactory, + injector + ); + + attachIndex( + retVal, + "tblWnulls.parquet", + new LongDimensionSchema("c1"), + new StringDimensionSchema("c2") + ); + + // {"col0":1,"col1":65534,"col2":256.0,"col3":1234.9,"col4":73578580,"col5":1393720082338,"col6":421185052800000,"col7":false,"col8":"CA","col9":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXZ"} + attachIndex( + retVal, + "allTypsUniq.parquet", + new LongDimensionSchema("col0"), + new LongDimensionSchema("col1"), + new DoubleDimensionSchema("col2"), + new DoubleDimensionSchema("col3"), + new LongDimensionSchema("col4"), + new LongDimensionSchema("col5"), + new LongDimensionSchema("col6"), + new StringDimensionSchema("col7"), + new StringDimensionSchema("col8"), + new StringDimensionSchema("col9") + ); + + return retVal; + } + + @Test + public void windowQueryTest() + { + try { + final String query = getQueryString(); + final String results = getExpectedResults(); + + testBuilder() + .skipVectorize(true) + .sql(query) + .queryContext(ImmutableMap.of("windowsAreForClosers", true, "windowsAllTheWayDown", true)) + .expectedResults((sql, results1) -> { + Assert.assertEquals(results, results1); + }) + .run(); + } + catch (Throwable e) { + log.info(e, "Got a throwable, here it is. Ignoring for now."); + } + } + + @Nonnull + private String getQueryString() throws IOException + { + return readStringFromResource(".q"); + } + + @Nonnull + private String getExpectedResults() throws IOException + { + return readStringFromResource(".e"); + } + + @Nonnull + private String readStringFromResource(String s) throws IOException + { + final String query; + try (InputStream queryIn = ClassLoader.getSystemResourceAsStream("drill/window/queries/" + filename + s)) { + query = new String(ByteStreams.toByteArray(queryIn), StandardCharsets.UTF_8); + } + return query; + } + + private void attachIndex(SpecificSegmentsQuerySegmentWalker texasRanger, String dataSource, DimensionSchema... dims) + throws IOException + { + ArrayList dimensionNames = new ArrayList<>(dims.length); + for (DimensionSchema dimension : dims) { + dimensionNames.add(dimension.getName()); + } + + final File tmpFolder = temporaryFolder.newFolder(); + final QueryableIndex queryableIndex = IndexBuilder + .create() + .tmpDir(new File(tmpFolder, dataSource)) + .segmentWriteOutMediumFactory(OnHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema(new IncrementalIndexSchema.Builder() + .withRollup(false) + .withDimensionsSpec(new DimensionsSpec(Arrays.asList(dims))) + .build() + ) + .rows( + () -> { + try { + return Iterators.transform( + MAPPER.readerFor(Map.class) + .readValues( + ClassLoader.getSystemResource("drill/window/datasources/" + dataSource + ".json") + ), + (Function) input -> new MapBasedInputRow(0, dimensionNames, input) + ); + } + catch (IOException e) { + throw new RE(e, "problem reading file"); + } + } + ) + .buildMMappedIndex(); + + + texasRanger.add( + DataSegment.builder() + .dataSource(dataSource) + .interval(Intervals.ETERNITY) + .version("1") + .shardSpec(new NumberedShardSpec(0, 0)) + .size(0) + .build(), + queryableIndex + ); + } +} diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaScanWindow.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaScanWindow.sqlTest index c3702985064..e5d8584827e 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaScanWindow.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaScanWindow.sqlTest @@ -1,4 +1,4 @@ -type: "failingTest" +type: "operatorValidation" sql: | SELECT @@ -7,6 +7,7 @@ sql: | page, LAG(page, 1) OVER (PARTITION BY "user" ORDER BY __time) as priorPage FROM wikipedia + WHERE countryIsoCode IN ('AT', 'JP', 'AU', 'UK') expectedOperators: - { type: "naiveSort", columns: [ { column: "user", direction: "ASC" }, { column: "__time", direction: "ASC" } ]} @@ -18,6 +19,252 @@ expectedOperators: outputColumn: w0 offset: -1 -# Not correct: there should actually be results for this query. But when the query runs, at the time I write this, it -# does not return any results at all. Therefore, we leave expectedResults empty and mark the test as a failingTest. -expectedResults: [ [ "to be added later, when fixing this test" ] ] +expectedResults: + - [ 1442055495852,"1.121.160.49","Walker, Texas Ranger","" ] + - [ 1442056175946,"1.121.160.49","Walker, Texas Ranger","Walker, Texas Ranger" ] + - [ 1442062719426,"1.121.160.49","Richard Norton (actor)","Walker, Texas Ranger" ] + - [ 1442053449011,"1.123.133.249","Inside Out","" ] + - [ 1442030282870,"1.123.149.111","Christina Ricci","" ] + - [ 1442062306734,"1.129.96.160","Andrew Hastie (politician)","" ] + - [ 1442052408695,"1.129.96.219","Dorrigo Steam Railway and Museum","" ] + - [ 1442049229239,"1.129.96.242","Dorrigo Steam Railway and Museum","" ] + - [ 1442046541914,"1.152.96.170","Ayrton Senna","" ] + - [ 1442056892106,"101.128.232.140","清水浩","" ] + - [ 1442057005310,"101.128.232.140","清水浩","清水浩" ] + - [ 1442033283609,"101.140.3.146","水原茂","" ] + - [ 1442036282693,"101.162.15.184","The Biggest Loser (U.S. TV series)","" ] + - [ 1442036494525,"101.162.15.184","The Biggest Loser (U.S. TV series)","The Biggest Loser (U.S. TV series)" ] + - [ 1442053239190,"101.162.15.184","The Biggest Loser (season 3)","The Biggest Loser (U.S. TV series)" ] + - [ 1442094543511,"101.162.15.184","The Biggest Loser (season 3)","The Biggest Loser (season 3)" ] + - [ 1442054030928,"101.162.192.189","Second Keating Ministry","" ] + - [ 1442054439219,"101.162.192.189","Second Keating Ministry","Second Keating Ministry" ] + - [ 1442062617292,"101.180.202.153","2015 AFL season","" ] + - [ 1442042291553,"103.18.40.202","Wikipedia:Auskunft","" ] + - [ 1442030546957,"103.20.255.191","Same-sex marriage in New Zealand","" ] + - [ 1442028937717,"103.225.121.237","AKB48握手会傷害事件","" ] + - [ 1442025765045,"103.56.218.196","욱일기","" ] + - [ 1442018949298,"106.131.175.193","正岡大介","" ] + - [ 1442061568629,"106.132.207.6","白石涼子","" ] + - [ 1442074511372,"106.150.80.109","ノート:アズ熊谷","" ] + - [ 1442022640886,"106.161.56.138","山口敬之","" ] + - [ 1442061180690,"106.188.137.190","救急救命士養成所","" ] + - [ 1442061491224,"106.188.137.190","救急救命士","救急救命士養成所" ] + - [ 1442100695886,"106.188.187.22","Petronas Towers","" ] + - [ 1442033017165,"106.188.237.163","ダウンロード違法化","" ] + - [ 1442019638402,"110.1.177.43","ユーロビート","" ] + - [ 1442039765889,"110.1.177.43","虹の都へ","ユーロビート" ] + - [ 1442097432832,"110.20.158.121","Flavia Pennetta","" ] + - [ 1442097158335,"110.20.234.69","Yaoi","" ] + - [ 1442101346090,"110.20.234.69","Yaoi fandom","Yaoi" ] + - [ 1442022181729,"110.21.43.182","Planet Hollywood Resort & Casino","" ] + - [ 1442054156682,"110.22.186.92","Aris F.C. (Thessaloniki)","" ] + - [ 1442060921634,"110.32.28.203","Politics of Réunion","" ] + - [ 1442095172937,"110.67.252.181","Shinya","" ] + - [ 1442073188151,"111.216.35.85","うしおそうじ","" ] + - [ 1442070741096,"113.197.250.14","フィンテック","" ] + - [ 1442065282546,"113.36.127.77","福岡県立修猷館高等学校の人物一覧","" ] + - [ 1442020617093,"113.38.112.244","エフエム東京","" ] + - [ 1442043701284,"114.148.67.27","オウム真理教の修行","" ] + - [ 1442057752095,"114.149.159.81","鮎川誠","" ] + - [ 1442030463066,"114.149.98.37","2015年全米オープン (テニス)","" ] + - [ 1442036595334,"114.161.17.122","京都放送","" ] + - [ 1442085773238,"114.163.215.4","コンコルド墜落事故","" ] + - [ 1442019941523,"114.165.9.29","架空の乗り物一覧","" ] + - [ 1442073082802,"114.167.235.83","ランドール・パーク","" ] + - [ 1442036882364,"114.168.102.41","杉浦圭子","" ] + - [ 1442046282828,"114.186.112.218","オイルショック","" ] + - [ 1442030866019,"115.64.101.40","Snow leopard","" ] + - [ 1442030416128,"115.69.63.229","Ibrahim Sunday","" ] + - [ 1442046588882,"117.18.154.84","969運動","" ] + - [ 1442022015086,"117.55.68.51","セルゲーエフ","" ] + - [ 1442022520647,"117.55.68.51","セルゲーエワ","セルゲーエフ" ] + - [ 1442063719891,"117.74.50.110","ソード・ワールド2.0リプレイ from USA","" ] + - [ 1442039556838,"118.106.18.200","今夜もオトパラ!","" ] + - [ 1442049541582,"118.106.18.200","青森放送","今夜もオトパラ!" ] + - [ 1442047677181,"118.109.38.229","ケイゾク","" ] + - [ 1442049350391,"118.109.38.229","ケイゾク","ケイゾク" ] + - [ 1442063820782,"118.109.38.229","ケイゾク","ケイゾク" ] + - [ 1442080232899,"118.111.38.174","外傷初期診療ガイドライン日本版","" ] + - [ 1442033322693,"118.238.237.196","推理小説","" ] + - [ 1442090528892,"119.104.157.254","おかしな二人","" ] + - [ 1442075598708,"119.104.36.170","パスポート","" ] + - [ 1442038707947,"119.170.41.230","スーパーマン (1978年の映画)","" ] + - [ 1442031883720,"119.25.37.181","六甲人工島線","" ] + - [ 1442018890048,"119.26.148.34","青野武","" ] + - [ 1442046705300,"119.26.148.34","クッキングパパの登場人物","青野武" ] + - [ 1442058062503,"119.26.148.34","林先生が驚く初耳学!","クッキングパパの登場人物" ] + - [ 1442059524102,"119.26.148.34","エンジェル・ハート","林先生が驚く初耳学!" ] + - [ 1442065445099,"119.26.148.34","本田望結","エンジェル・ハート" ] + - [ 1442068465888,"119.63.112.86","DIABOLIK LOVERS","" ] + - [ 1442069164239,"119.63.112.86","GANGSTA.","DIABOLIK LOVERS" ] + - [ 1442091530861,"119.63.112.86","BS-TBS独占放送深夜アニメ枠","GANGSTA." ] + - [ 1442030131807,"119.72.245.161","The VISIONALUX","" ] + - [ 1442047214542,"120.147.6.7","African Union Mission to Somalia","" ] + - [ 1442032176994,"121.102.43.4","利用者‐会話:Yasuyuki Takeo","" ] + - [ 1442066932287,"121.117.180.156","Ulmendorf","" ] + - [ 1442023340480,"121.208.92.28","Moreton Bay","" ] + - [ 1442050348689,"121.211.56.55","Dominique Dunne","" ] + - [ 1442100154443,"121.211.82.121","List of Harry Potter characters","" ] + - [ 1442048444641,"121.223.129.191","List of dinosaur genera","" ] + - [ 1442027637932,"121.45.214.80","Total Drama Presents: The Ridonculous Race","" ] + - [ 1442054470231,"121.93.47.91","チキンハート","" ] + - [ 1442069124156,"121.93.97.147","機甲創世記モスピーダ","" ] + - [ 1442095062216,"122.108.156.100","United States presidential election, 1892","" ] + - [ 1442098671579,"122.108.156.100","United States presidential election, 1892","United States presidential election, 1892" ] + - [ 1442023533650,"122.25.10.78","星のカービィ! も〜れつプププアワー!","" ] + - [ 1442024788039,"122.25.10.78","陸上防衛隊まおちゃん","星のカービィ! も〜れつプププアワー!" ] + - [ 1442072416878,"122.25.10.78","黒鷺死体宅配便","陸上防衛隊まおちゃん" ] + - [ 1442074382146,"122.25.10.78","ロリコンフェニックス","黒鷺死体宅配便" ] + - [ 1442072607975,"123.100.63.20","User:MichelleFranklin/AustralianSkeptics","" ] + - [ 1442084541627,"123.217.1.75","JOIN ALIVE","" ] + - [ 1442056682107,"123.223.224.148","相馬康一","" ] + - [ 1442056818652,"123.223.224.148","杉崎亮","相馬康一" ] + - [ 1442057183557,"123.223.224.148","日高里菜","杉崎亮" ] + - [ 1442037245734,"124.148.87.34","List of Doctor Who audio plays by Big Finish","" ] + - [ 1442038279908,"124.148.87.34","List of Doctor Who audio plays by Big Finish","List of Doctor Who audio plays by Big Finish" ] + - [ 1442039505649,"124.148.87.34","List of Doctor Who audio plays by Big Finish","List of Doctor Who audio plays by Big Finish" ] + - [ 1442051202571,"124.148.87.34","List of Doctor Who audio plays by Big Finish","List of Doctor Who audio plays by Big Finish" ] + - [ 1442027954979,"124.169.165.146","Talk:Peter O'Toole","" ] + - [ 1442096498423,"124.169.17.234","Coca-Cola formula","" ] + - [ 1442046696410,"124.170.55.214","Northern Football League (Australia)","" ] + - [ 1442049310216,"124.171.44.159","Liberal Democratic Party (Australia)","" ] + - [ 1442035151510,"124.188.44.154","List of Doctor Who audio plays by Big Finish","" ] + - [ 1442092615464,"124.45.73.228","鹿児島県立加治木高等学校","" ] + - [ 1442023710710,"125.168.85.156","Beatrix Potter","" ] + - [ 1442050705594,"125.192.149.97","ドカチン","" ] + - [ 1442063450156,"125.198.182.94","交響曲第5番 (チャイコフスキー)","" ] + - [ 1442036586983,"125.2.29.54","バトルスピリッツ 烈火魂","" ] + - [ 1442020773422,"125.205.166.177","粉河町","" ] + - [ 1442070671364,"126.10.25.69","民王","" ] + - [ 1442065314043,"126.120.17.52","ガンズ・アンド・ローゼズ","" ] + - [ 1442057530890,"126.144.8.136","小柳友貴美","" ] + - [ 1442057729757,"126.144.8.136","張込み","小柳友貴美" ] + - [ 1442058237893,"126.144.8.136","篠原涼子","張込み" ] + - [ 1442058403152,"126.144.8.136","瀬戸朝香","篠原涼子" ] + - [ 1442058944024,"126.144.8.136","顔田顔彦","瀬戸朝香" ] + - [ 1442059192896,"126.144.8.136","山中聡","顔田顔彦" ] + - [ 1442064154338,"126.29.44.172","アンフェア the answer","" ] + - [ 1442046633404,"126.64.184.114","中山道","" ] + - [ 1442045829839,"130.54.130.227","静岡空港","" ] + - [ 1442056451492,"137.147.143.241","Royal Australian Air Force","" ] + - [ 1442020256428,"137.147.165.107","Michael Sharkey","" ] + - [ 1442045368765,"139.218.187.26","National Youth Competition (rugby league)","" ] + - [ 1442021132744,"14.201.22.221","2015 Roger Federer tennis season","" ] + - [ 1442078715198,"14.203.177.143","Captain (cricket)","" ] + - [ 1442063042912,"153.170.91.23","一色氏","" ] + - [ 1442040686108,"153.200.114.68","芋たこなんきん","" ] + - [ 1442042536291,"153.200.114.68","ごちそうさん (2013年のテレビドラマ)","芋たこなんきん" ] + - [ 1442042555091,"153.200.114.68","ごちそうさん (2013年のテレビドラマ)","ごちそうさん (2013年のテレビドラマ)" ] + - [ 1442024836364,"153.207.13.117","リーガ・エスパニョーラ","" ] + - [ 1442055615954,"153.230.62.16","刺客 聶隱娘","" ] + - [ 1442079228153,"161.202.72.168","Shaun King (activist)","" ] + - [ 1442080592078,"161.202.72.168","Talk:Shaun King (activist)","Shaun King (activist)" ] + - [ 1442082673917,"161.202.72.168","Talk:Shaun King (activist)","Talk:Shaun King (activist)" ] + - [ 1442054785180,"180.17.65.26","88式鉄帽","" ] + - [ 1442064037343,"180.27.228.108","虎ノ門事件","" ] + - [ 1442092840431,"180.54.251.210","鹿児島県立加治木高等学校","" ] + - [ 1442026399564,"182.168.99.4","ミャンマー","" ] + - [ 1442080053112,"182.173.142.88","ドクタースランプ","" ] + - [ 1442080079822,"182.173.142.88","ドクタースランプ","ドクタースランプ" ] + - [ 1442071861338,"188.23.31.111","Diskussion:Löwengrube (Serie)","" ] + - [ 1442041218594,"202.174.58.102","テッド (映画)","" ] + - [ 1442043359400,"202.174.58.102","立川流 (密教)","テッド (映画)" ] + - [ 1442050346906,"203.112.37.97","ABCフレッシュアップベースボール","" ] + - [ 1442037360618,"203.87.72.77","Hakoah Sydney City East FC","" ] + - [ 1442033433557,"210.128.82.94","列車種別","" ] + - [ 1442059451406,"210.132.145.235","キン肉マンスーパー・フェニックス","" ] + - [ 1442028766085,"210.145.159.111","小桜エツコ","" ] + - [ 1442047728381,"210.159.186.209","ハウス食品","" ] + - [ 1442073460368,"210.160.37.25","沖野晃司","" ] + - [ 1442073648703,"210.160.37.25","沖野晃司","沖野晃司" ] + - [ 1442025498133,"210.171.30.75","車塚 (伊丹市)","" ] + - [ 1442053077021,"210.194.52.87","小田急多摩線","" ] + - [ 1442079405551,"210.199.238.238","ズバリ!当てましょう","" ] + - [ 1442084605765,"210.199.238.238","インド","ズバリ!当てましょう" ] + - [ 1442048760081,"211.0.97.62","アフガニスタン紛争 (1978年-1989年)","" ] + - [ 1442058690158,"211.1.73.194","筑波山","" ] + - [ 1442066152552,"211.135.251.28","FNSスーパースペシャルテレビ夢列島","" ] + - [ 1442090789576,"212.95.7.43","Pamela (Vorname)","" ] + - [ 1442093822018,"212.95.7.57","Eşkıya Dünyaya Hükümdar Olmaz (dizi)","" ] + - [ 1442052594311,"213.102.112.181","Diskussion:Schlacht um die Krim","" ] + - [ 1442055194535,"213.102.112.181","Diskussion:Schlacht um die Krim","Diskussion:Schlacht um die Krim" ] + - [ 1442055264461,"213.102.112.181","Diskussion:Schlacht um die Krim","Diskussion:Schlacht um die Krim" ] + - [ 1442087241623,"213.240.105.56","16. Mai","" ] + - [ 1442073949064,"218.110.166.206","エゴグラム","" ] + - [ 1442048197658,"218.227.98.24","福岡空港","" ] + - [ 1442031289907,"218.45.13.57","サムライエッジ","" ] + - [ 1442043817524,"219.102.143.16","ヒルナンデス!","" ] + - [ 1442076175461,"219.106.24.95","中村れい子","" ] + - [ 1442076262194,"219.106.24.95","阿藤快","中村れい子" ] + - [ 1442076443347,"219.106.24.95","西岡善信","阿藤快" ] + - [ 1442067732777,"219.106.70.167","アンフェア the end","" ] + - [ 1442047016164,"219.110.114.95","永田久光","" ] + - [ 1442039078679,"219.110.28.111","こちら葛飾区亀有公園前派出所の登場人物","" ] + - [ 1442039059890,"219.112.132.106","野口茂樹","" ] + - [ 1442039204914,"219.112.132.106","野口茂樹","野口茂樹" ] + - [ 1442039593604,"219.112.132.106","井場友和","野口茂樹" ] + - [ 1442042116370,"219.112.132.106","高村祐","井場友和" ] + - [ 1442043631086,"219.112.132.106","岩本勉","高村祐" ] + - [ 1442033187371,"219.127.27.114","あんハピ♪","" ] + - [ 1442033425162,"219.127.27.114","林田理沙","あんハピ♪" ] + - [ 1442034666225,"219.127.27.114","八木沼純子","林田理沙" ] + - [ 1442026079559,"219.127.82.26","Wikipedia:サンドボックス","" ] + - [ 1442047519920,"219.161.34.177","小川虎之助","" ] + - [ 1442088730524,"219.171.105.167","2015 FIFA U-20ワールドカップ","" ] + - [ 1442090473730,"219.171.105.167","AFC U-19女子選手権2015","2015 FIFA U-20ワールドカップ" ] + - [ 1442098469728,"219.182.78.3","宮里久美","" ] + - [ 1442029124664,"219.198.137.251","逆転イッパツマン","" ] + - [ 1442048505685,"219.51.50.31","突撃!ナマイキTV","" ] + - [ 1442053430316,"219.51.50.31","突撃!ナマイキTV","突撃!ナマイキTV" ] + - [ 1442075782897,"219.67.166.28","オリンピア・ミラノ","" ] + - [ 1442035985727,"220.109.120.102","ワールドサッカー ウイニングイレブン 2016","" ] + - [ 1442036083342,"220.210.178.20","櫻井よしこ","" ] + - [ 1442024997507,"220.212.136.113","国際地学オリンピック","" ] + - [ 1442024799232,"220.213.14.136","甲府市中心市街地活性化基本計画","" ] + - [ 1442025695124,"220.213.14.136","放置自転車","甲府市中心市街地活性化基本計画" ] + - [ 1442026228735,"220.213.14.136","甲府市中心市街地活性化基本計画","放置自転車" ] + - [ 1442051686890,"220.62.54.3","坂上忍の成長マン!!","" ] + - [ 1442044538982,"221.170.137.161","週末キッズタイム","" ] + - [ 1442053936462,"221.77.244.149","ボーダーブレイク","" ] + - [ 1442101230028,"222.12.9.108","ネプリーグのクイズ・企画一覧","" ] + - [ 1442046505197,"222.148.228.64","おじゃる丸のエピソード一覧","" ] + - [ 1442046590842,"222.148.228.64","おじゃる丸","おじゃる丸のエピソード一覧" ] + - [ 1442051878307,"223.132.42.72","THE IDOLM@STER CINDERELLA GIRLS ANIMATION PROJECT","" ] + - [ 1442027068163,"49.128.148.160","パンク・ファッション","" ] + - [ 1442035102420,"49.181.161.0","Hazelbrook railway station","" ] + - [ 1442059510611,"49.183.149.106","2015 AFL finals series","" ] + - [ 1442045663055,"49.199.44.74","List of fictional European countries","" ] + - [ 1442047899247,"49.199.44.74","List of fictional European countries","List of fictional European countries" ] + - [ 1442051573911,"49.199.44.74","List of fictional European countries","List of fictional European countries" ] + - [ 1442051658029,"49.199.44.74","List of fictional European countries","List of fictional European countries" ] + - [ 1442055279567,"58.106.135.42","Newcastle Jets FC Youth","" ] + - [ 1442059004452,"58.111.155.139","Stadium Australia","" ] + - [ 1442048316558,"58.156.119.18","特盛!よしもと 今田・八光のおしゃべりジャングル","" ] + - [ 1442028373987,"58.156.158.18","ソーラーパネル","" ] + - [ 1442046515015,"58.174.137.134","Jarryd Hayne","" ] + - [ 1442055231108,"58.175.186.6","Animals in Islam","" ] + - [ 1442056110424,"58.175.59.60","Queer Lion","" ] + - [ 1442040037037,"58.183.124.153","ジュラシック・パークIII","" ] + - [ 1442046294050,"58.188.28.158","小山薫堂の“温故知新堂”","" ] + - [ 1442051739818,"58.188.28.158","平塚競技場","小山薫堂の“温故知新堂”" ] + - [ 1442058986459,"58.188.28.158","2015年ワールドカップバレーボール","平塚競技場" ] + - [ 1442065595186,"58.188.28.158","ザ・ラストヒロイン〜ワルキューレの審判〜","2015年ワールドカップバレーボール" ] + - [ 1442065723868,"58.188.28.158","癒し屋キリコの約束","ザ・ラストヒロイン〜ワルキューレの審判〜" ] + - [ 1442068144671,"58.188.28.158","地域発ドラマ","癒し屋キリコの約束" ] + - [ 1442068557997,"58.7.81.140","Nose gunner","" ] + - [ 1442056051721,"58.70.174.162","トヨタ・エスティマ","" ] + - [ 1442042617971,"58.89.8.10","津軽信枚","" ] + - [ 1442038469110,"58.93.38.113","ウエルシア薬局","" ] + - [ 1442072948002,"59.166.42.154","コミック乱ツインズ","" ] + - [ 1442034973871,"59.86.97.35","パズドラZ","" ] + - [ 1442071403042,"59.87.28.229","История создания сил специальных операций Армии США","" ] + - [ 1442072375155,"59.87.28.229","Армия США","История создания сил специальных операций Армии США" ] + - [ 1442027556018,"60.224.133.167","Torres Strait Islanders","" ] + - [ 1442018825474,"60.225.66.142","Peremptory norm","" ] + - [ 1442043839445,"61.200.121.189","コズミックフロント","" ] + - [ 1442036042205,"61.245.69.54","テレビ朝日系列日曜昼前の情報番組枠","" ] + - [ 1442063039021,"62.178.213.52","Cimburgis von Masowien","" ] + - [ 1442068687074,"77.80.5.199","Santiago Niño Becerra","" ] + - [ 1442088415400,"83.65.220.213","Philomena Höltkemeier Story Consulting","" ] + - [ 1442093778448,"90.146.48.55","Z Nation","" ] + - [ 1442088395645,"91.115.65.157","Al-Masdschid al-Harām","" ] diff --git a/sql/src/test/resources/drill/window/datasources/allData.csv b/sql/src/test/resources/drill/window/datasources/allData.csv new file mode 100755 index 00000000000..c00dd8fdc37 --- /dev/null +++ b/sql/src/test/resources/drill/window/datasources/allData.csv @@ -0,0 +1,11196 @@ +-337516559,39342852852629160,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.418,1952-08-14,false,729363085.95,8:16:8.58 +406158122,81588677006971200,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.418,2001-03-08,false,1292460500.48,9:11:49.17 +1221407024,30009558124347168,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.419,2000-10-18,true,395110006.277,18:44:25.43 +-1609141704,47841997008600128,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.420,1991-05-13,true,1293582041.37,20:52:8.56 +-1032159521,38891661529640288,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.420,1965-02-21,false,983657842.924,19:46:10.42 +-414035709,37344878123283896,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.420,2003-09-03,false,466674349.327,15:21:34.39 +1694834300,81923062562407680,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.421,1991-06-20,false,474713471.17,10:31:55.24 +-1874845950,4496901140547891,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.421,1958-10-17,false,889911941.945,1:37:47.31 +877901888,31452794467889696,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.421,1966-06-19,false,1189761583.32,1:50:5.11 +847222588,66263082908851272,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.421,1971-09-26,true,482511936.897,11:28:25.1 +-1889383968,65973293780212808,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.421,1991-06-09,false,472880577.743,6:12:12.40 +-708074788,71705740868271344,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.422,1975-10-20,false,1048713254.74,9:38:51.10 +-1457190846,53209680683077360,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.422,1992-05-18,false,147932888.44,20:2:5.20 +1086311701,78366091242119200,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.422,1968-06-06,true,936437541.786,20:50:52.8 +1907158860,13276876335203164,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.422,1954-10-10,false,1091096591.83,20:26:42.34 +114272140,10866869904043878,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.422,1960-05-12,false,371555243.082,2:2:58.3 +-830394984,67285760676197944,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.422,2006-10-25,true,1426805339.42,18:31:10.32 +-1094584211,71184555316985264,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.423,1980-07-06,false,390513711.473,4:28:38.23 +-1954269479,23663159493540352,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.423,1976-03-26,false,972747639.559,17:32:57.7 +424095942,50279291136262144,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.423,1959-05-24,true,1134302618.55,2:33:58.28 +-350143576,53078979319492352,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.423,1972-10-24,false,47019775.7462,11:58:35.46 +-1162827816,26493206551509832,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.423,2012-04-23,true,268051680.205,2:22:46.39 +360722582,24047259207853364,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.423,2013-04-09,false,384895861.757,19:22:43.32 +-1679149645,90604964771508720,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.424,1954-07-01,true,273055641.297,19:44:28.9 +-1212232159,50537587258210752,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.424,1975-06-21,true,401229124.414,1:34:9.49 +-222037402,57581801621486000,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.424,1999-09-22,false,184965459.559,8:18:52.57 +-348723399,74449884062386400,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.424,1973-10-22,true,1241097485.87,11:13:31.56 +562181214,53734770394172456,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.425,1978-03-07,true,1095310441.42,10:52:41.45 +-1215442209,7873304607399414,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.426,2005-08-05,false,264747268.761,7:46:55.29 +1864574763,92176649228931520,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.426,1952-09-11,true,44580966.2418,10:50:6.13 +2126671066,10228472499454832,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.426,1973-05-03,true,1252568155.5,2:30:18.34 +732598839,10390879559649690,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.427,1993-06-19,true,868835049.003,4:47:53.57 +1245184587,61382128415557504,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.427,1984-03-25,false,530376804.39,22:42:37.18 +-683491299,38451518463823096,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.427,1966-08-18,false,26960747.6481,7:46:11.3 +118711345,87029844461312688,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.428,1982-10-24,true,911002550.279,5:41:50.14 +-956862531,21040574161052100,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.428,2013-02-04,true,924694540.11,12:49:7.13 +957912690,18643204878089392,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.428,2008-08-27,false,125727344.595,8:12:26.56 +-1476499048,36157941841357520,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.429,1950-03-18,false,1279099100.34,14:16:48.7 +2101806553,86022880197719744,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.429,1965-10-07,false,1178899337.49,14:45:36.17 +-1285626232,36220772560158160,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.429,2008-10-22,true,420766450.283,17:21:17.17 +1285020079,67010430927702336,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.429,1997-11-23,true,949884963.943,4:39:12.38 +-1684479922,13614712512498402,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.429,2013-09-16,false,1112084945.5,7:10:6.55 +1964196285,84274620765606560,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.430,1960-06-11,false,954531854.701,2:8:14.5 +773871907,41295272389399832,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.430,2001-09-20,false,1315605568.86,6:6:39.4 +-1672685276,15055139993663150,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.430,1953-06-22,true,936204255.77,10:22:43.3 +-129564479,51566198676862192,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.430,2000-01-21,false,1177480656.47,3:28:2.49 +-1753065881,72813785820639104,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.430,1993-10-16,false,1422997879.08,5:50:46.30 +1577015858,46735957714817640,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.430,1982-05-09,true,1027651127.57,11:19:21.56 +-85773523,69668119021283352,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.430,1965-05-13,true,102635426.051,4:45:11.38 +1364787899,27077365269499548,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.431,1955-06-08,true,1125729860.97,2:17:41.23 +-2044690449,51483497774827872,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.431,1968-05-10,false,422305438.068,5:4:15.48 +-820906821,11380987114305648,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.431,1990-07-17,true,1267122931.61,22:49:49.30 +55445062,58847932469368144,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.431,1953-04-04,true,830455866.482,11:56:39.36 +845698702,37230917095145712,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.431,1964-01-19,false,45422860.5029,2:1:16.57 +386733894,24096397934717308,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.431,1989-11-06,true,763454104.971,13:57:47.14 +-631896228,60383759737041520,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.431,2006-01-19,true,1129483174.85,8:5:35.16 +1100434619,62551584333708880,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.432,1999-05-12,false,69583187.3006,7:49:56.30 +1012831770,21600191230112492,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.432,1991-01-14,true,727978345.49,3:44:50.30 +1480911016,71557386759755464,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.432,1986-06-16,true,828874169.301,15:16:47.11 +252456332,33487621463398648,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.432,1961-01-12,true,399985456.569,20:28:42.39 +1798220069,62690507401848376,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.432,1993-07-08,true,1422757320.68,3:44:25.17 +2081228257,17710541344395470,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.432,1958-10-12,true,123097632.842,7:42:22.43 +-768561899,10717244532504126,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.432,1950-03-10,false,913633367.303,21:36:1.24 +725356309,75536887780882768,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.433,2004-02-06,true,61660275.6464,2:21:10.13 +1134069281,45247194738415224,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.433,1975-06-11,true,1402389808.45,1:32:7.40 +726826191,75773581890206400,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.433,1999-06-19,true,291155129.129,22:21:2.27 +1333005522,6945957647182316,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.433,1963-02-03,false,727735064.287,16:19:46.26 +1970523294,85275081484331584,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.433,1987-08-16,true,1419780808.66,13:46:22.5 +894233766,80723159532444944,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.434,2011-05-26,true,568905792.53,9:45:18.46 +-1775207427,85057015634888048,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.434,1957-05-03,true,466736977.42,6:23:1.11 +-1883525775,27191232848000716,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.434,1986-06-02,true,267962270.836,7:52:5.45 +545781329,62107794343808064,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.434,1990-11-22,false,622820332.764,7:50:38.35 +-963439824,49746162141333424,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.434,2000-09-23,true,1248389308.76,16:57:41.26 +-1740459210,22024616041311632,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.434,1962-06-04,false,144804901.984,16:38:18.37 +346418118,49360209911451976,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.434,1989-03-27,true,856574532.391,10:27:16.44 +-401278146,27066193254615236,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.434,1958-10-22,true,735715255.558,14:24:2.56 +558510930,19219411094639400,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.435,1958-08-21,true,891015086.626,19:44:24.45 +2028594285,21043887252922656,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.435,1962-10-04,true,882104473.692,15:15:14.28 +-965621377,48449744005058168,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.435,1963-09-06,true,66771489.57,17:33:15.25 +-1084275727,41400076730694872,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.435,1951-07-11,false,1373276708.68,16:18:10.17 +1261094481,78073864403533984,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.435,2006-01-18,true,585965409.481,1:14:31.22 +1848234209,45638802269638424,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.435,1952-11-07,false,1018840199.49,14:15:39.58 +-882881117,82460843979534976,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.435,1977-09-19,false,508935424.91,14:31:9.24 +-1009846176,54748128142408760,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.435,1954-05-10,false,642346608.687,14:50:44.20 +-1620084062,74172056813519776,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.436,2004-01-03,true,1382373099.46,18:34:57.16 +1902726619,50249449395343312,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.436,1959-10-04,false,80259100.9467,2:47:19.35 +1556127935,8015346857876060,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.436,1958-02-03,true,132584910.067,8:3:53.34 +3735294,43456077486288592,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.436,1972-02-03,false,30651483.2189,2:46:35.49 +-184379150,33989780061133908,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.436,2009-11-23,false,1297005483.13,18:31:44.51 +1851408268,29592918347430820,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.436,1972-09-13,false,1022892671.24,3:7:28.45 +-629975387,91593866328701680,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:02.436,1967-08-16,false,142409041.462,11:39:36.16 +114212107,18722982535746364,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.436,2010-10-14,false,791697316.619,14:47:4.8 +638994389,38736512265938640,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.437,1965-02-03,true,1004850762.17,6:21:11.35 +1516905426,32165800534356192,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.437,1978-07-22,true,1025241466.4,15:39:42.32 +-2042792190,37109423862185288,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.437,1991-07-15,false,802561700.03,18:34:41.1 +48073256,62663018796171120,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:02.437,1970-03-09,false,330283711.488,20:28:57.33 +-1401732385,56605991984182312,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.437,2007-02-18,false,218199000.393,22:8:22.4 +-956155329,77853698221557984,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:02.437,1969-10-02,true,1382620548.33,8:11:53.36 +-667574628,51362504868610592,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:02.437,2008-07-02,true,196167371.6,8:58:32.31 +117614719,65833806095121720,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.187,2009-05-04,false,17077787.1774,2:27:41.41 +1871593409,59537104592147512,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.187,1979-11-23,false,1214512072.76,17:9:42.23 +1848641615,78682312368260880,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.188,1951-11-11,false,811212847.813,7:20:14.58 +-1229481959,2940637841987563,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.188,1963-06-19,false,729061713.092,20:7:44.53 +-1243010331,70790999736578488,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.189,1969-03-07,true,426606343.36,1:48:29.16 +-1992087006,39440406683987304,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.189,1978-01-16,true,431494753.597,1:21:9.34 +356408271,72980644289736160,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.190,2012-07-03,true,1371662506.92,10:19:49.26 +-642420403,33891996980596060,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.190,2003-03-05,true,472823622.951,17:19:20.17 +-1403799044,80573140259687072,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.191,1961-10-07,true,229327319.939,13:39:28.52 +424636954,15787711661436580,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.191,1956-11-19,true,1275804359.9,20:23:50.11 +-1129624027,20997052548303956,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.191,1992-06-02,true,516098581.852,15:35:49.38 +938795427,51269930888197400,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.192,1973-02-16,true,566225240.621,17:20:46.45 +1651962819,78237389190206736,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.192,1968-11-07,true,1074207367.06,17:39:15.34 +1953850205,55907528531786896,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.192,1956-07-16,false,1132616900.75,2:20:21.48 +394548832,15194345929344082,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.192,1957-10-21,false,1093351597.27,9:24:54.4 +-667166544,40321677002893672,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.193,1959-09-03,true,775583043.571,20:51:44.26 +1749513102,50396906951596952,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.193,1957-04-03,true,574191508.56,11:45:18.16 +574741554,51143101787181992,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.193,1958-02-25,true,1283137314.42,6:46:9.38 +1347762872,29627322288957788,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.193,1957-02-08,true,846878009.988,19:27:51.43 +1799722833,53455116676876880,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.193,1968-01-26,true,1199502181.88,10:13:15.33 +-525186159,3117821681321933,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.194,1956-02-17,false,230933430.625,3:2:18.52 +-883541662,59787566526751728,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.194,1972-09-07,true,767452867.576,12:3:11.40 +421290448,53032497515759832,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.194,2007-03-03,true,357679712.575,14:12:8.10 +1440961443,32262990251954004,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.194,1989-01-23,false,1258995456.36,6:43:1.40 +167345857,7283190439999611,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.194,1977-07-25,true,1122853356.32,17:8:51.47 +142642652,48088849781105760,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.194,1964-10-21,false,145556820.868,4:40:53.49 +-22945986,83904532111206320,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.195,1953-11-18,true,1406916779.27,8:24:11.47 +1614981412,78790672067243440,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.195,1965-11-03,false,153076037.961,20:39:37.32 +2037816805,77227427867246464,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.195,1999-07-27,false,1399180273.13,20:35:38.38 +706486214,7256417956233616,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.195,1957-04-08,true,1166420567.71,10:18:11.22 +679199002,70221883528736608,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.195,1996-05-25,true,1043672804.75,11:53:39.15 +584107408,4006642976191560,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.195,1984-05-25,true,1433022341.88,14:43:13.48 +-594703625,27585801017593448,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.195,1952-04-27,true,1203882782.41,18:32:23.33 +-1743092932,13493798137282428,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.196,1990-01-02,true,727799137.324,9:26:54.23 +-656904416,83619025833658288,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.196,1958-01-20,false,1182204776.24,19:50:17.40 +-1342988952,5150790424148193,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.196,2013-06-22,true,1179076320.5,15:38:11.53 +-12706018,47451013436464752,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.196,1966-10-15,true,1250883255.09,13:1:17.7 +-1944941884,2493457460913019,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.196,1954-06-21,false,532572168.053,7:12:41.25 +932143361,67007779764078944,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.196,1996-05-15,false,36639687.9516,5:32:8.58 +993208297,50578261577327976,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.196,1995-08-22,true,1227503154.02,12:38:53.2 +-125742992,65681883743728784,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.197,1993-02-05,false,1107868008.36,19:25:52.7 +1636068547,83700216599369968,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.197,2004-05-25,true,351692065.606,22:28:37.54 +1168812666,38521081409019144,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.197,1979-07-22,false,1295536909.38,2:32:25.26 +892047968,32827342154293596,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.197,2005-01-20,true,699133716.006,14:46:46.1 +-1808020828,62548158909677816,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.197,2004-01-16,false,82685662.5279,2:54:46.35 +-416190128,49403955089561552,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.197,2007-05-27,true,1252188537.2,19:14:36.34 +-2082360870,87999787401938704,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.197,1983-02-18,false,794838744.443,8:18:40.53 +-1942607483,55168105938898392,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.198,1969-06-07,true,494173286.435,21:19:31.35 +-1920012328,50990920124747528,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.198,1995-08-27,false,284955357.439,10:13:51.4 +-1876222793,73446492162851488,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.198,1967-09-27,false,749267550.107,21:38:17.28 +-1803537392,77439949048002448,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.198,1981-09-07,true,490521142.327,21:55:22.1 +-566283742,26189448892841524,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.198,1998-01-04,true,797079268.617,3:41:35.27 +-2086414876,89634644400962784,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.198,1995-09-23,true,698833854.443,15:55:37.23 +-357023663,9143564837037538,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.199,1996-08-02,true,280214549.474,15:48:1.23 +-1858725655,55894844570121296,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.199,1992-01-18,false,1123297446.84,22:15:31.46 +1071111583,90465049545302064,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.199,1988-09-18,true,708144508.27,1:47:1.22 +-74046298,65309421384888760,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.199,1976-10-03,true,446222909.437,22:21:30.15 +-1629642733,14279606805587692,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.199,2010-10-02,false,364781798.408,4:36:31.29 +1232104188,66973747832568168,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.199,1955-10-05,true,772495826.984,12:30:18.55 +261775141,2392948826283254,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.199,2012-10-22,false,1237261205.19,4:3:19.9 +-111557029,60699432999640272,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.200,1959-05-16,false,1140171059.68,22:21:12.24 +1287658193,23377338829177160,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.200,1990-10-04,true,26751215.5327,17:48:9.14 +-855965778,1308632127204208,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.200,1995-04-21,false,1100344381.53,5:15:42.20 +-1360153772,41409022460035328,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.200,2009-07-15,true,6518254.58302,4:29:37.53 +-767914818,79577100230773152,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.200,1984-10-09,true,930991816.33,21:51:20.9 +-1983167901,79697922868375616,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.200,1960-05-17,true,676600235.284,5:48:32.13 +-1584507033,49819598142137608,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.200,1969-11-18,true,389067470.815,11:56:27.52 +69511610,90530648677526368,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.201,1955-11-25,false,1036532983.24,14:57:55.34 +1057571888,69389196008575840,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.201,1989-06-11,false,827993577.385,10:7:12.30 +-1013593903,8696452946815642,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.201,1986-01-27,false,546072125.366,11:41:18.49 +-1952289525,90314650182792304,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.201,2002-11-08,false,831599547.161,9:15:25.11 +363907907,91933036106867808,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.201,1963-11-13,false,649147678.632,20:34:42.48 +574564647,22528930022668840,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.201,1999-04-26,true,168156874.303,14:26:21.36 +-1561609275,62479548173774576,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.201,1961-11-23,false,1322606861.18,22:50:54.53 +-1833056794,67668054165341168,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.202,1986-02-15,true,1107532983.84,2:37:8.57 +523152928,85244696429901408,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.202,1987-01-24,true,456515334.136,19:1:47.53 +2092399987,63464411378671624,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.202,2004-09-04,true,448641645.313,12:6:44.16 +-586084349,74777316096675808,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.202,2009-01-06,true,466898565.536,5:30:22.25 +262225060,45031303571096864,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.202,1980-01-21,false,1172394967.56,14:54:24.46 +2062572048,24019809179374348,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.202,1961-11-15,false,817035203.848,2:53:55.40 +111920480,40326758988287312,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.202,1974-07-09,false,935263018.551,5:47:20.25 +949355235,27349099792568648,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2014-06-02 00:28:12.203,1984-10-04,false,1274594628.86,17:52:32.48 +1940445085,24351433094447260,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.203,1972-04-05,true,496019050.96,7:23:50.3 +-1161936725,76530184542389856,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.203,1955-03-02,true,960251649.638,2:39:5.52 +135278082,34162397764423488,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.203,1987-08-20,true,499037665.645,14:24:39.47 +786447103,65127376952167208,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.203,2002-06-14,false,22051073.7456,4:25:52.40 +1301464280,60045382802880656,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.203,1955-03-19,true,703320152.245,15:22:1.35 +-1994221469,5669019205958595,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.203,1996-04-09,true,865448532.309,4:52:3.27 +1939052867,10129155948968612,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.203,1964-04-21,true,441624877.124,13:45:15.54 +-72952361,65513905819910056,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.204,1999-09-05,false,203797592.934,5:32:49.26 +-1188923179,7028167308442112,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.204,1999-06-16,true,39022345.7914,7:27:24.52 +-1221973252,85305788280217280,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.204,1976-06-08,false,1428698572.04,13:2:49.43 +763007785,89348367649732992,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2014-06-02 00:28:12.204,1996-05-16,false,337066606.108,8:51:5.41 +-1639446363,18131794666370672,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.204,1964-09-23,true,1084533841.51,20:17:37.17 +-1936313357,36054090470107176,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.204,1961-04-17,false,1394786823.36,3:31:5.33 +598718226,74551785694859184,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2014-06-02 00:28:12.204,1969-07-21,false,659154306.44,3:1:2.25 +-301831473,63584629495426160,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.205,1996-10-25,false,712604511.708,15:20:46.7 +-1699967528,85935777101468688,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.205,2003-06-21,false,774018153.742,20:9:56.3 +808172993,87282757784552320,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2014-06-02 00:28:12.205,1975-10-25,false,91100965.2582,2:7:9.4 +-581378712,22713251086092144,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.210,1996-03-13,true,1127206554.75,22:46:58.15 +-1199077564,55895329106297512,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.211,2008-10-01,false,1188097669.13,15:12:40.35 +-897452244,78652179172382224,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.212,2006-09-08,false,696671244.054,12:17:17.55 +-1205022429,81982417508663040,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.212,1987-05-23,true,721391390.095,17:19:33.45 +943194732,75564576854228656,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.212,1975-03-11,true,137896878.677,17:42:50.57 +-589815268,57628077523770120,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.212,1996-11-16,false,1308440050.61,16:44:18.2 +1767428106,86794881027020160,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.213,1995-02-03,true,1119156912.5,10:20:5.4 +-696018244,82242599242198432,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.213,1976-03-15,false,1082140910.82,5:31:24.55 +1786532679,49306790427073024,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.213,1997-04-02,true,1270827642.53,21:49:18.12 +104199260,85622401650406048,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.213,1990-03-16,true,1049231735.74,2:31:3.14 +603974176,92013866710763936,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.213,1969-04-15,true,1304632922.71,22:31:33.50 +-1251781405,37841316061314888,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.213,1987-09-01,false,1326663870.12,22:47:11.46 +-666773478,70843912132424304,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.214,1969-11-16,true,1429465015.67,9:3:40.9 +-1928767465,84406260340856800,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.214,1960-04-15,false,310001115.904,15:51:44.44 +229800855,1689928755479142,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.214,1977-05-07,false,907671170.725,1:48:54.16 +-104219238,88257560307758832,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.214,1958-01-02,true,361970164.222,18:19:7.36 +904531746,25810587848379332,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.214,2011-09-17,false,556722061.247,18:37:14.11 +940536651,30968087393117972,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.214,2006-01-23,false,474679461.123,13:22:49.19 +-582678795,78707923788812176,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.214,1995-03-04,true,280818526.954,10:29:6.42 +1739339500,64939010050071776,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.215,1956-02-16,true,441556222.913,6:7:37.41 +-1996550317,12519079921918372,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.215,1965-07-16,false,59768213.0201,15:5:4.16 +166445501,29129809662578340,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.215,1972-08-17,true,814696424.579,16:18:36.20 +-1984611014,90859571430708480,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.215,1951-08-04,true,636706320.147,5:34:46.31 +1943387326,41054439828650688,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.215,1953-04-09,true,827109222.707,21:45:11.4 +134002769,88122555321285760,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.215,1957-08-12,true,1227815537.56,5:16:54.2 +-1460029776,14941147945409904,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.215,1992-10-25,false,144642354.127,22:6:58.18 +-773075736,67978972944239328,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.216,2004-10-27,false,1251782974.07,8:20:42.34 +2458455,3556895768037601,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.216,1958-06-23,false,1432361259.89,18:33:46.17 +1285675730,38478003229474632,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.216,1997-11-19,true,252459626.077,19:24:42.12 +-800142147,71458885222768160,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.216,1959-03-27,true,956601435.532,17:12:3.1 +41235409,69574663547907872,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.216,1973-06-22,false,1099810150.91,15:35:23.37 +-853971595,83290202377453792,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.216,2006-11-25,true,1037022673.81,6:19:23.14 +-517591201,41508342131608632,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.216,1985-02-05,true,251041798.883,18:31:6.33 +-1279907676,29553176269584856,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.216,1998-11-20,false,52389361.6705,7:47:29.17 +1185207016,23026058054305200,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.217,1982-07-09,false,966891056.508,3:53:14.47 +-1909244626,63020319439176320,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.217,1980-01-10,true,228675017.344,20:46:19.27 +-1354882117,45898299724940456,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.217,1960-11-10,false,166536924.366,10:57:55.10 +607432327,91965781392729392,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.217,2009-07-11,true,229938806.773,12:58:58.7 +-2007229564,87918937289054064,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.217,1953-05-04,true,463515857.107,12:40:3.50 +948350413,23845812174200740,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.217,1967-07-16,false,433791927.286,22:54:4.9 +-1261817433,75389609846948992,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.217,1951-11-25,false,977367592.305,16:46:9.45 +-2051495059,67550738121414056,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.217,1965-04-18,false,1354048681.52,2:7:31.42 +664450986,79750515148569936,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.218,1999-09-27,true,850556411.598,6:17:15.25 +391286906,34492717640510516,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.218,1950-09-11,false,1014621240.97,5:58:42.27 +-711495031,22869695746668156,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.218,2004-05-06,false,5207196.06412,8:1:42.8 +-941493239,15873368156625112,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.218,2004-02-09,true,250074180.382,10:24:49.50 +-748665695,84563383243765648,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.218,1969-03-13,false,949122890.159,20:20:35.19 +-612485701,72997226305804128,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.218,1986-06-12,false,982771396.484,8:8:24.7 +582185187,11761453010605158,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.218,1979-07-11,true,1059059499.84,22:23:56.21 +-30534607,19486397130968424,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.219,2000-11-14,false,904764035.101,5:26:3.36 +-586053528,78864541897986528,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.219,1973-03-07,true,1104961421.8,22:49:35.28 +1805807608,77459948134724016,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.219,1975-11-13,false,1012424901.82,19:22:37.14 +-681851476,12707538807040748,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.219,1965-08-16,true,1302240695.26,2:13:7.29 +-1845996474,20000342296750532,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.219,2007-03-11,true,1273610586.65,20:17:33.24 +-1921364939,72880309936785776,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.219,1964-08-16,false,806450400.111,6:7:49.45 +1638795257,5501639144450222,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.219,2008-01-11,false,1144685149.9,2:4:48.13 +-571093711,7689936846692321,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.220,1985-06-27,true,591167338.407,3:56:56.28 +-319095089,8400516342324152,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.220,1971-11-07,false,24505472.2012,10:14:50.28 +1512851709,38294403610357600,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.220,1999-08-15,true,377155056.124,6:19:51.13 +-338977918,21298373997559708,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.220,2008-11-27,true,140166361.511,2:18:16.47 +1630392792,1443342254050109,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.220,2005-07-15,true,305468119.678,15:4:44.10 +1146794276,29486999579049668,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.220,2007-08-06,true,268726830.352,15:42:54.5 +1269685367,5817776160390851,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.221,2003-11-22,true,807776720.102,9:47:26.28 +-1454495465,74960970286894640,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.221,1960-01-23,true,917021603.026,19:44:42.32 +-191131431,18921815323657892,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.221,1983-09-22,true,949346931.877,22:41:39.39 +202358675,27304314306611128,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.221,1998-02-22,true,1071524343.88,12:35:50.1 +1682607448,60452077150287088,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.221,1976-04-09,true,1029653598.16,8:3:45.51 +-977244593,74334722844430688,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.222,2008-04-12,true,1217956839.19,22:47:11.40 +269064038,45082864704444536,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.222,1986-01-14,true,317423396.243,8:15:37.14 +571676400,37128558030078712,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.222,2006-07-18,true,1129922783.78,1:16:29.12 +1401645707,74726397530016464,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.222,2008-08-05,true,57688391.777,9:24:53.34 +1720883548,85637434210683568,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.222,1976-01-09,true,776441147.111,8:1:49.2 +505300168,38535820824029432,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.222,1995-02-16,true,704571876.448,14:53:12.32 +1276010126,69850058768595440,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.222,1982-10-18,true,922826039.884,22:23:39.49 +1366245409,13574073282423296,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.223,1957-04-10,false,319343224.191,22:23:30.22 +-1775322274,37041336553639272,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.223,1977-07-09,false,161232980.344,1:27:42.43 +1796261920,81899167778996832,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.223,1955-10-14,true,821229270.428,22:31:42.38 +1749798597,58461880749416784,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.224,1987-08-13,false,528989027.624,10:24:38.32 +1533719798,40850862210414992,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.224,1998-09-12,true,512582533.807,15:11:3.37 +1166367135,24431915212234968,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.225,2001-10-14,true,758533982.393,16:38:57.36 +1421013938,59857001822347256,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.225,1994-08-25,false,956796499.463,13:38:22.11 +618766373,80528888603968928,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.226,2005-10-25,true,382799423.65,7:45:58.10 +-1068702721,44659880640000336,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.226,1969-08-05,true,210900664.351,21:25:58.7 +-1142907168,22875945424925952,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.226,1985-06-14,true,673115124.583,3:46:34.21 +1657865648,79876466364616352,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.226,1988-04-10,false,959938280.079,3:46:46.18 +2071325721,86594291956107056,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.227,1968-08-07,false,398258671.397,3:26:29.10 +-1937968547,38387433731601560,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.227,2003-04-15,true,1136568139.62,14:28:15.36 +388506205,1296756603059507,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.227,2007-02-07,false,931018654.114,4:41:11.15 +609304125,82891602088800848,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.227,1969-02-08,false,1006117892.55,10:16:3.27 +119644480,48103019545491008,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.227,1989-06-01,true,241029864.116,16:19:47.23 +-529290306,72652637908316240,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.228,2002-09-07,true,793691526.092,8:11:37.15 +-1529448688,86959175397027632,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.228,1997-02-22,true,734381390.575,18:13:12.51 +2021550580,79208062587942800,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.228,1970-11-23,true,1220089613.76,7:51:52.19 +-1632862226,83814118292710768,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.228,1987-02-01,true,1094177717.52,10:39:55.58 +2037751982,9265970944823102,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.228,2002-02-05,false,637929367.774,18:16:50.12 +-938741185,86785929090741328,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.228,2004-06-10,true,860844635.62,3:10:27.19 +2062916075,53921195374588696,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.229,1957-01-25,true,1143452086.29,10:19:15.35 +-1501427412,40103676941225536,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.229,2010-01-21,false,1219024314.34,7:57:57.43 +-545066320,28196563840392520,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.229,1992-07-08,false,1224742019.25,20:33:37.46 +824834755,44249330752249096,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.229,1982-09-02,false,14062772.881,13:5:11.46 +637465557,34825883884947488,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.229,2009-04-18,false,766414349.081,11:47:41.16 +1750464256,52852329990326848,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.229,1989-10-24,false,1247355280.28,6:5:45.32 +-1883740536,40509827650770880,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.229,1967-04-01,false,1186393300.97,22:58:20.10 +959494217,67929213176266128,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.230,2007-03-05,false,351633663.836,4:36:58.26 +1864256513,29016505358828564,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.230,1966-04-08,true,1228108695.62,8:7:6.48 +382850840,56451174453470896,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.230,1982-04-24,true,1084929814.46,19:37:11.37 +-685678216,52707618089952592,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.230,1964-05-01,false,315332370.672,15:9:25.49 +-1346434845,20638575844598508,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.230,1961-06-21,false,406661122.487,15:24:54.47 +-1783011708,24713979194910608,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.230,2009-03-10,false,797708183.703,2:2:4.9 +852541341,55883553189987640,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.230,2003-06-23,true,276681500.56,20:26:6.54 +1039989909,46790471410692600,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.231,1972-02-06,true,1003739121.57,12:31:5.4 +2105192762,6352715291481395,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.231,1977-06-12,false,1259392574.27,22:19:14.25 +-1541166111,10109601474454150,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.231,1967-07-02,true,704879024.221,20:25:1.22 +-2112130527,57276219346643528,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.231,1959-07-03,true,581317884.86,10:15:18.40 +-481469853,2878307290593771,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.231,1984-09-06,true,1139588086.34,7:11:42.58 +954502035,31113384070273188,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.231,1956-08-02,false,177487909.249,14:1:54.38 +-1130374570,83975520509474208,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.231,2003-10-12,false,63429420.0891,2:15:23.51 +-1090858718,15042261439933174,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.232,2007-01-08,false,534072629.18,1:13:14.48 +-459796169,43548477117446992,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.232,1997-04-27,true,585781984.232,10:37:23.57 +1116059189,74718842077036848,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.232,1999-03-23,true,645956704.447,13:53:40.4 +110034084,64151148336616688,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.232,1968-05-14,true,954229833.201,18:20:38.58 +1006857490,58560685317818184,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.232,1968-02-02,false,1108080160.93,22:8:40.43 +-899245397,58129693716540120,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.232,1992-05-14,false,1203396377.59,7:11:18.8 +23226585,19878698738482380,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.232,1981-01-26,true,192248594.522,22:18:25.17 +-1346042433,80984575529745200,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.232,1989-06-04,true,840206511.739,1:36:44.25 +-1157839737,17806915891656786,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.233,1988-10-03,true,315303938.44,22:42:54.35 +-185447984,52789401269276728,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.233,1967-10-22,true,638572343.159,19:21:54.18 +1954450766,34917827395928780,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.233,1963-08-16,true,257099926.37,2:20:44.1 +481475641,72698410551016048,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.233,1971-07-23,true,1253503402.94,6:10:47.31 +1998442205,24290178397135616,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.233,1972-09-10,false,1432840826.89,2:10:41.30 +-966546565,72448283682935936,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.233,2013-07-19,true,1388725747.59,1:53:33.21 +1878904326,14484165884931082,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.233,1996-10-13,false,284896291.967,16:39:2.58 +-718571512,78886948231143584,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.234,1953-04-04,false,1268747874.88,1:38:57.22 +1834193853,41051485464532744,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.234,2010-05-08,true,539874580.433,2:18:38.52 +-460874852,90895998911604976,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.234,1971-06-24,false,952769299.828,11:1:32.18 +710659261,53496795142059152,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.234,1958-03-19,false,771675673.592,13:43:24.16 +-490453909,2019037681473556,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.234,1961-05-23,false,140037953.94,22:44:27.34 +1382143774,88487447595063584,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.234,2011-05-13,false,1284469163.1,4:9:13.8 +-991364043,65430595946902048,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.234,2006-02-08,true,411793139.352,5:34:3.17 +656698357,20511002690816380,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.235,1983-02-15,false,1193529575.6,13:20:47.41 +784134800,3285341939309199,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.235,1975-02-26,false,1082023168.27,13:47:33.51 +60414829,3426004226878064,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.235,1950-07-17,true,115337617.702,18:49:35.46 +1398676564,84630906563885504,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.235,1986-05-27,false,872783935.013,20:35:11.9 +-708655326,35173711936198116,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.235,1965-03-17,false,772893755.217,16:43:50.6 +-547427650,25542722934942352,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.235,2007-06-06,true,434295580.68,16:5:41.37 +-377564035,24549069986657476,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.235,1978-10-07,false,722867403.539,5:19:53.33 +-151879518,29720255713920584,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.236,1986-11-02,false,1368004269.4,8:50:43.20 +-1855209224,18183739702254716,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.236,2005-03-16,true,1390366387.39,22:10:49.44 +-99585803,50508186397830360,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.236,2009-03-16,true,1199006376.78,11:3:14.40 +1925823800,55601019327064360,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.236,1972-08-25,false,1432465428.74,9:38:14.1 +36919597,44101958054605864,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.236,1965-07-16,true,772831073.628,9:50:34.42 +11880810,63329376601585248,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.236,2009-03-17,false,20139032.6193,18:31:6.45 +1367111692,91434723221518608,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.236,2001-05-24,false,64208432.796,22:31:54.42 +-110132518,73120725821748928,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.237,1983-05-24,false,352572802.693,15:41:54.12 +-367641613,77013349276134400,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.237,1987-11-06,false,468150893.614,18:27:54.37 +281705295,18626432601204684,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.237,1985-05-04,false,849329102.236,4:15:22.2 +1823572553,22265082101714760,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.237,1968-06-20,true,9876684.23655,12:35:42.28 +26698950,30355153698458952,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.237,1964-04-14,false,714126953.705,15:36:27.55 +-1819822263,41392951162875912,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.237,2002-01-07,true,26349183.1151,6:50:45.52 +-893725740,42745878501232112,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.237,1951-03-12,false,1417490907.49,12:8:32.41 +-686739051,77933496444267936,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.237,1996-07-06,false,654267541.561,13:50:44.4 +67159074,33699671325539360,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.238,1951-06-23,false,1267878091.06,21:2:20.3 +-1206363689,4383043082516029,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.238,1958-02-24,false,718475912.738,1:45:15.15 +-1305382925,57726103092157496,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.238,1979-10-05,true,351772388.907,18:56:5.5 +-895184836,80610512774512528,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.238,1973-10-19,true,412121519.311,6:11:45.16 +734826992,72543424437493040,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.238,2013-04-13,false,231096767.396,17:31:34.36 +-661967764,56071543782585712,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.238,1992-02-21,true,776539852.173,7:37:48.48 +-1653147036,3494899520831037,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.238,2005-03-26,true,491250239.438,12:47:48.8 +-543875496,38671444465313288,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.239,2003-03-10,false,255353216.493,1:44:48.2 +104513543,23579478003171840,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.239,1956-03-22,false,882766502.489,21:15:25.20 +1840437470,701562594164510,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.239,1956-09-10,true,1152027041.01,13:45:47.14 +-115760873,65799033684197592,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.239,2012-11-09,false,833823704.781,11:38:32.15 +-1981088315,59014536317739120,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.239,1982-02-06,false,766229465.987,4:28:44.52 +-707942697,83190992500592464,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.239,1978-08-16,true,48599661.7881,12:38:51.44 +1131446057,8899750625211361,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.239,1965-09-22,false,277768146.422,13:13:20.22 +-915178555,76676347647721712,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.240,2007-08-16,true,538155585.195,16:21:48.33 +-581208513,40998041562958488,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.240,2003-02-01,false,796362049.578,6:23:27.54 +2119545777,35447473354092576,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.240,1973-11-05,false,303314873.394,8:25:7.13 +-1181769167,63872462799142848,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.240,1988-10-16,true,168596916.605,20:36:1.44 +674236281,2242304034518210,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.240,1983-09-02,true,1262463735.44,8:37:16.22 +1526619654,24999867033023232,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.240,2007-04-01,false,445728505.816,20:21:22.56 +1661595035,35704405957025720,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.240,1976-02-12,true,1425149040.65,18:4:23.57 +1248932042,50760143306999448,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.241,2003-07-08,true,150465993.829,5:5:17.32 +-1924062931,49420011443388528,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.241,1970-10-15,true,743501152.98,14:5:51.26 +926526340,19251766995428876,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.241,2004-11-20,true,962346876.52,18:10:30.42 +755142144,6522524371223501,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.241,1956-05-17,false,730112243.664,2:55:11.31 +793599574,39822631466625400,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.241,1997-03-09,false,195508858.206,7:51:3.25 +-138242337,89593047470925824,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.241,1964-08-02,true,685836823.25,22:1:34.4 +-165564310,47082820879480376,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.241,1984-06-05,false,737459664.498,1:51:43.55 +-626014586,89479629377805952,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.242,2004-06-11,true,528531954.413,1:37:18.23 +-2050112160,81414690864866432,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.242,1995-08-24,false,526377238.428,20:29:46.54 +-1668927298,1499478485084272,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.242,1990-07-23,false,837680781.062,19:38:55.5 +-1467282678,28236661118714472,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.242,2000-06-08,false,112103465.78,18:48:16.18 +-1998543281,2970062426283909,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.242,1982-06-03,true,427982711.101,10:9:24.57 +340142722,91296914565653888,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.242,1964-07-20,true,285801616.444,5:9:55.50 +1364646706,25677923492867132,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.243,1952-04-07,false,208253768.714,10:1:31.49 +1191651254,87647536084758000,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.243,2001-11-19,false,475497564.266,1:40:41.42 +401034970,14295094710043854,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.243,1980-11-19,true,373791809.615,2:49:44.31 +1635017760,79976064098739056,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.243,1970-10-06,false,176715340.292,10:49:30.5 +46391009,61973924490418088,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.243,2012-04-19,true,1213930404.24,16:6:32.41 +781713075,49295047678251376,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.243,1981-06-12,false,863401398.277,6:23:54.50 +-1694463395,76116780570980256,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.243,2010-04-20,true,491065647.743,21:17:17.38 +271576829,75090165486385056,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.243,1964-06-17,true,303216761.3,3:51:18.35 +1415676978,45991687123848216,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.244,1954-11-10,false,152090832.371,7:36:44.43 +1716308851,83280288463698880,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.244,1979-09-19,false,872105565.633,4:10:25.56 +1985848050,15063842108807732,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.244,2002-06-21,false,1034183871.54,6:34:56.54 +1216837717,68024023391501064,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.244,1956-04-20,true,1009165349.66,17:40:15.29 +838150780,61653542857344096,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.244,2003-03-26,true,70078190.0055,3:54:55.33 +1601273462,89754926425558992,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.244,2004-02-12,false,861925570.244,20:3:24.19 +-1452032668,21728242778528808,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.244,1968-02-25,true,266270396.34,16:27:13.42 +1573695633,47389730912362416,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.245,1959-07-21,false,789489151.665,22:52:11.25 +2034781814,81466861194028608,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.245,1974-05-15,true,133860797.89,12:1:15.57 +1793716879,75225778920344208,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.245,1954-11-03,false,269947365.515,9:55:52.53 +-1125929485,56888818462002088,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.245,1989-08-22,true,209585300.067,6:12:7.53 +-1602474864,23193304242584064,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.245,1965-11-14,true,1092749807.56,16:55:37.58 +-1993605852,44805507755755696,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.245,1990-02-11,true,688444989.592,13:32:36.5 +854479221,69355406294687040,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.245,1981-04-03,true,177907798.793,10:22:50.28 +-1210592314,35559174955367812,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.246,1971-06-15,false,389566140.436,11:42:38.56 +1387353629,7944237321061253,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.246,1954-09-09,false,1332854315.04,18:28:1.35 +1099930548,90735617600208752,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.246,1971-11-07,true,427072265.735,9:44:30.39 +-1528489661,71103558273779784,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.246,1977-11-21,false,559056828.119,22:11:41.46 +-2035448814,24248975345216348,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.246,2003-08-10,true,1156496406.63,20:57:6.41 +1330131746,28230297368604684,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.246,2013-11-03,false,319566586.688,22:6:32.38 +-1083360595,60993924739399128,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.246,1992-07-10,false,616131454.193,5:41:5.30 +2027950178,22227910648084204,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.247,1971-11-12,false,1384248982.89,6:41:10.10 +-1159409473,61418491208097896,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.247,1985-05-03,true,1274950455.08,9:58:39.31 +1826203944,76000900493150704,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.247,1953-10-10,false,196440317.742,15:22:23.1 +-1127577169,2963426315868447,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.247,2010-11-02,false,706806553.398,15:36:3.51 +879352444,27984302098949540,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.247,1981-03-02,true,1377477625.56,21:57:49.18 +159446745,74566815333190240,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.247,1999-07-13,false,387682126.476,22:28:58.38 +489960427,25475651320468264,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.247,1963-06-10,true,790487529.819,12:6:2.1 +1085820180,20435407642318584,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.248,1960-10-16,true,1236429203.96,17:13:49.58 +-1717028867,24381904877604096,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.248,1963-02-08,false,695832722.829,7:2:11.24 +-1782404444,53121639508334960,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.248,2006-03-25,true,279562561.017,3:6:21.34 +-1739741046,46607493438105264,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.248,1995-09-20,false,584557830.13,19:5:18.13 +620287564,65254251365666016,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.248,2005-08-27,true,407369625.856,12:8:17.38 +-666929266,33685317558003936,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.248,1999-04-19,true,158585277.667,11:14:12.39 +1040600275,62705428329592168,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.249,1985-11-09,false,1003239661.57,14:23:30.24 +-1667586560,34715481099850692,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.249,2004-11-06,true,664409828.99,2:10:51.17 +1060283080,79832169141816640,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.249,2004-08-19,true,1367734032.27,1:6:54.52 +1166877693,59351146972958144,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.249,1975-03-23,true,1203540637.34,18:25:37.51 +-1264238399,2531153862770944,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.249,1953-06-04,false,384138395.138,3:32:46.7 +-208469756,17352441721782958,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.249,1969-09-20,false,765163355.574,4:28:6.41 +2006091764,18618555477892576,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.250,1996-09-25,false,202941023.751,12:54:15.12 +377840927,37918338318650728,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.250,1993-01-12,true,865409593.11,14:55:38.17 +-1501290281,91911169794115312,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.250,1960-04-13,false,878033482.24,10:36:32.29 +768332233,49694733823177544,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.250,2008-01-19,false,714429594.113,12:13:55.6 +297883624,6166803298633871,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.250,2002-03-18,false,1064606676.03,14:17:31.17 +-205715715,57682819645949888,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.250,1968-10-04,true,48113160.2999,11:48:24.20 +-387766137,82916343160676512,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.251,1989-09-08,false,1405630760.64,1:1:30.6 +487236837,76713173912276848,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.251,1966-03-11,false,742816845.698,6:21:48.11 +309070217,50824143620187584,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.251,2012-08-14,false,335414841.124,11:39:23.21 +540898878,47624643708008856,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.251,1996-01-21,false,251388614.51,16:14:6.33 +1390460598,46290183816953704,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.251,2000-03-08,false,1026964019.68,21:47:3.35 +411979577,66556485997545440,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.251,1977-04-14,false,1042530605.96,4:9:17.35 +-1822245642,67650349394647536,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.252,2010-02-06,true,630168580.772,3:32:43.26 +772342403,76540077388210288,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.252,1988-10-01,false,709869066.997,11:39:16.27 +1648195401,7456699230457088,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.252,2003-07-20,false,624500230.748,12:28:16.49 +-1733509862,54770064305182616,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.252,1960-08-02,false,630017176.368,15:32:58.42 +228616432,21526572699937876,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.252,1966-05-25,true,727268536.795,2:23:10.11 +341532410,38218502273502784,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.252,1976-04-14,false,1119669724.81,2:37:8.46 +346661918,64207188457458208,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.252,2006-01-16,true,1088206675.96,3:21:22.30 +192489475,11565967688471634,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.252,1992-06-25,true,59476594.8431,20:28:55.53 +-92796158,59964437183367736,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.253,1957-11-26,false,47162498.5058,4:24:32.1 +-140163011,88484742731222944,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.253,1984-04-14,false,118529435.286,14:33:27.18 +-864943448,35714522238159484,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.253,1988-10-18,false,1331233568.63,17:6:35.33 +275750005,47521151889782592,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.253,1975-04-02,true,274944660.55,7:47:56.34 +468191895,85221099938134240,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.253,1985-09-14,false,1269168722.3,14:33:5.32 +-938326625,73519081505251520,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.253,1963-08-07,true,1048896765.14,4:30:16.31 +-1495932519,63455248665486392,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.253,1999-08-18,true,344341364.216,12:34:15.25 +2104320221,40452769440609160,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.254,1982-05-04,true,1391649983.07,19:6:11.51 +-1993322650,23660675362909800,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.254,1974-11-12,true,861927057.253,17:58:7.9 +2103932920,24694768002962648,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.254,1985-02-10,true,584380511.047,2:58:18.48 +963716412,34780523157733244,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.254,1996-08-12,false,152626928.296,19:11:21.56 +-2673337,71810728748243384,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.254,1972-08-10,false,74922486.5046,16:51:50.31 +-773560232,39128926962655912,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.254,1973-11-13,false,112973037.553,2:33:38.5 +-175403361,43607375200992424,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.255,1965-03-25,false,283398968.468,2:5:19.24 +-1297365256,87876646550898400,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.255,1986-11-04,true,1350363171.25,2:8:37.5 +781151090,67452763585955408,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.255,1991-01-15,false,1386503120.19,4:2:43.8 +1496795835,19030892908926044,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.255,1961-10-25,false,1382671340.55,22:29:34.29 +-1984577925,78733078018088544,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.256,1985-07-02,true,737105634.601,19:11:30.58 +989180561,14757918377302272,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.256,1983-11-15,true,739801583.12,6:43:23.48 +657605090,36528662216461536,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.256,1994-10-01,true,1373440594.29,17:29:53.25 +52871504,30285046300415600,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.256,2005-07-10,false,502947363.373,17:3:26.6 +201455625,16040139575484714,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.257,1957-03-05,true,1296057404.59,1:8:51.57 +-559376413,13928708387665388,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.257,1992-11-23,true,589925110.506,1:57:14.20 +66009179,28276145604215472,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.258,2005-06-17,false,1334467989.38,4:26:40.15 +-19020624,58698459397470240,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.258,1979-11-07,true,469135102.459,17:46:36.41 +1934818299,22790671729541152,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.258,1989-02-13,false,498848985.592,17:52:38.3 +328587926,921148918230343,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.259,2012-07-02,false,1345750996.35,1:45:24.33 +1930697537,47275820856921240,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.259,1978-11-12,true,1173968335.29,11:32:37.4 +1649863069,69159434184615080,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.259,1974-06-18,false,718865243.077,16:49:45.30 +-1774579564,53349041813719288,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.259,2000-05-08,true,995547300.729,22:34:36.44 +-559876261,86716422852693440,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.259,2012-07-02,true,625005882.203,22:25:3.56 +1356501459,20944743305964616,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.260,1979-10-09,true,1170689039.69,1:24:9.21 +-1785921524,75042059117491936,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.260,1951-07-14,true,266041977.823,12:51:44.6 +-1517933790,38927054721867336,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.260,1963-08-06,true,816327818.427,11:16:56.54 +1359502364,51087125595308512,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.260,1975-08-16,true,650674670.507,19:42:23.45 +1774855978,15903651628771656,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.260,1990-01-19,false,1364874810.95,4:36:2.12 +781709136,41391578926706240,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.260,1957-11-06,false,59893396.9312,10:55:34.35 +1989804252,81996798047439008,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.260,2004-09-23,false,1381361166.69,10:16:42.43 +-334378578,11920192676903608,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.261,1960-08-25,false,21182704.4801,19:48:26.47 +-1066798608,36759544271904320,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.261,1972-03-10,true,598629357.691,4:57:34.10 +85682045,12604959582907586,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.261,2000-11-07,false,1301332837.87,4:37:29.7 +-1595363596,66194544329130200,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.261,1952-03-05,false,1229252092.4,18:35:5.44 +-684527214,46879614581358728,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.261,1964-04-24,false,889221917.599,15:1:54.10 +238247744,76789981380296800,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.261,2000-07-09,true,253515927.516,14:27:35.44 +-1223406247,65639090545952424,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.262,2011-08-15,false,1302516837.43,14:20:20.14 +-2042684521,83917326568487248,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.262,1974-09-24,false,955062964.293,1:55:25.50 +1642025518,23954769623536444,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.262,2000-01-20,true,1368474065.84,17:45:29.46 +739010894,83718313092765120,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.262,1962-04-21,true,75053405.54,2:40:40.15 +632010333,43589454319063816,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.262,1976-03-24,false,348296807.656,7:10:8.9 +1008312787,11748474000111882,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.262,1975-02-08,false,730745722.071,6:16:11.21 +-1193321778,1987787951924213,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.262,1965-06-14,true,121821333.076,3:55:11.39 +-1315357088,9478056646903050,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.263,1974-08-11,false,1276490504.55,18:28:50.33 +1862477712,12039589319915520,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.263,1956-04-09,true,1091346175.61,13:51:39.14 +1701554249,79781731624948080,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.263,1984-07-08,true,1336983796.46,11:9:28.44 +1305918201,81949950128779792,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.263,1962-03-14,false,456196030.123,10:12:55.16 +-902570122,60745340957005456,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.263,1960-02-15,false,436588702.218,14:51:41.1 +-1601933651,19633398462068428,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.263,1976-01-23,false,768063282.668,2:24:33.2 +87629182,44598699733905344,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.264,1977-07-16,true,248715063.054,4:14:23.34 +-403959125,14329613925367246,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.264,1950-07-18,false,1294532450.42,16:3:32.16 +1604777610,66186110827839896,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.264,1951-03-14,false,834343059.473,9:35:19.10 +-1662232528,67289621317945520,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.264,1982-09-02,true,600056158.364,10:38:5.54 +-1334977236,7240189694349742,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.264,1994-08-23,true,1308365911.77,5:22:40.24 +1942263174,14089375984725268,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.264,2001-01-14,true,1094705320.69,15:41:22.19 +-210748726,9549426194802986,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.264,1961-03-25,false,176792174.569,21:8:6.42 +695832895,64699443966718816,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.265,2004-03-21,false,51866341.8891,3:47:23.24 +-519604181,9800994424498484,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.265,1984-03-04,false,295070178.697,3:31:39.24 +-257407146,39308309396969368,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.265,1965-03-20,true,638990670.266,9:27:28.24 +-224755206,70575726290547376,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.265,1996-11-03,true,392156452.548,7:51:27.40 +-2066942235,90032412746144128,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.265,1978-06-13,true,485194677.863,6:4:13.52 +-1831278659,22460876014930504,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.265,1963-01-06,false,1087065932.13,2:2:2.38 +-787380785,61834226413901992,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.265,1965-10-24,true,212130707.339,18:37:17.47 +-146849297,67156856901797384,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.266,1956-08-21,true,449540160.182,14:21:13.50 +-185004415,24033315150344500,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.266,1974-06-02,false,908505811.312,8:1:55.50 +-1069905336,3439259987344456,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.266,1959-03-04,false,747068616.167,6:31:58.3 +-2075162580,48921447042878024,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.266,1976-05-26,false,1270657491.89,3:14:7.47 +560536393,7272286501951857,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.266,1963-10-18,true,1200369624.06,20:24:42.27 +-466727933,13874844105432116,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.266,2001-06-23,false,1428631216.48,17:34:15.10 +659991730,53168754279467592,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.266,1972-09-19,false,378917345.457,14:6:31.5 +-931006910,26154041396002592,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.267,1959-02-23,false,637558309.3,22:14:55.20 +1217781610,43731048369121984,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.267,1968-11-07,false,1092953740.75,2:14:11.7 +1949191296,31309996279414796,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.267,1988-07-13,false,586337714.609,4:34:37.43 +2043172624,60740139617547960,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.267,1957-07-14,false,591084153.006,17:19:49.55 +479025617,69335860356336736,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.267,1970-08-06,false,1117448889.55,20:33:31.54 +-1901770193,35116106055334788,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.267,2011-07-21,false,364876359.669,13:30:33.29 +693128438,72747718588910976,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.267,2013-01-19,false,263237586.622,15:41:11.42 +-1549014763,45352541914042832,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.267,1975-06-09,false,1241755936.47,20:56:16.3 +-217913899,36238018696911256,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.268,1957-09-14,false,1301896792.59,6:5:12.45 +2039594183,27687420821827820,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.268,1994-06-13,true,1297645005.54,21:48:30.35 +-1542853463,69875690735798240,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.268,1995-06-18,false,149396190.533,19:47:37.2 +801288853,90758663131617648,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.268,1958-09-24,true,992247872.423,11:56:4.27 +-1610061736,46619078845335216,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.268,2004-10-26,false,500726447.185,3:7:24.48 +-651487397,1463339004143554,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.268,1999-01-26,false,407793353.643,11:14:46.26 +938629934,37619265949786880,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.268,1991-06-25,false,69069872.0732,18:51:7.53 +-436232972,63540420557819072,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.269,2013-03-18,false,446440776.565,19:57:51.5 +351925497,15379497129147106,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.269,2009-06-09,false,1023258512.78,11:55:36.35 +1115301153,30571176308909712,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.269,2006-05-01,false,1393448353.94,1:19:39.57 +493088340,7977515682143335,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.269,1967-06-07,true,625616218.202,19:36:32.17 +1395244831,89081136083491664,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.269,2010-03-23,false,1311796910.71,8:6:13.19 +-1564209783,6653902830397788,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.269,1999-01-10,true,19081310.5015,4:3:50.38 +1286527984,20124469967522776,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.269,2010-07-07,true,1109107355.32,7:6:45.19 +-714574087,36411410928345600,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.270,1988-01-16,true,1278551194.53,14:36:10.56 +1434109367,16430374149904692,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.270,1953-02-16,false,1313637537.03,9:8:12.42 +1139939453,57740746264944240,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.270,1986-09-13,true,224224595.649,18:11:25.20 +-1125644521,54549718911198048,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.270,1952-10-06,false,325373029.606,21:21:55.5 +3894967,44891360057242616,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.270,1997-04-08,true,1414403217.52,7:15:30.12 +443849949,63842689277657408,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.270,1999-07-09,true,125439504.404,12:25:12.42 +975736032,10235520720305982,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.270,1977-10-25,true,258999546.39,6:23:18.51 +490059672,12473709534770002,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.271,1986-07-17,false,1197734766.45,14:10:46.23 +739713009,10315145708888770,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.271,1992-02-25,false,794606171.211,5:30:20.46 +-218613437,36330941185308024,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.271,1998-09-06,false,907085515.185,7:3:6.55 +2075949217,66502198091446048,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.271,1973-08-23,true,274870534.681,6:13:19.17 +1268795371,41911032645913408,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.271,1993-11-02,true,178445922.089,12:19:43.20 +445079167,15656192069512172,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.271,1979-02-21,false,1239042084.05,16:42:14.21 +-1380965944,74881141917536816,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.271,1978-09-05,false,890417210.079,5:53:11.50 +-911296314,9068685671463824,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.272,1954-07-20,false,1384031930.46,22:6:41.22 +1627504699,39184014605567672,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.272,1996-09-10,true,626693755.052,14:23:20.55 +589177492,34426122564416840,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.272,1971-11-10,false,1123363167.83,17:42:45.58 +-31047575,29876919749109708,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.272,1968-09-19,false,1098021662.13,4:31:16.26 +-1980122300,68419126261302040,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.272,1967-05-06,false,1097024013.97,15:10:32.41 +1896903003,78791872934888688,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.272,1959-01-05,true,18953494.458,13:26:19.40 +77248650,52722963578551288,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.272,1974-02-01,true,1000750151.02,20:47:1.17 +-674022541,88845825317269744,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.273,1992-07-12,false,377105231.045,4:34:15.17 +-906725187,28093788554461808,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.273,1989-08-10,true,419578549.501,3:18:31.23 +-638028014,9962801653517824,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.273,1994-02-12,false,1329809122.44,11:45:40.16 +1452487082,85851352103798848,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.273,1984-05-09,true,82325169.2661,8:2:38.51 +-1129273078,14609636841798338,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.273,1953-03-08,false,329750029.426,12:24:31.28 +1487579871,38831593466810224,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.273,2008-05-05,true,1110178536.34,4:34:16.40 +1780866566,5443279670426296,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.273,2008-02-24,false,284236244.969,9:9:3.10 +2074130677,23269313427138900,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.274,1997-08-19,false,897420006.196,2:19:7.6 +1068837066,77937054371852304,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.274,1974-01-21,true,907361546.234,18:6:16.58 +-1345629342,27952065456313476,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.274,2008-02-09,true,124507536.657,1:34:47.38 +1343033775,4587650602149478,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.274,2008-01-10,true,1347735964.87,15:36:44.36 +2113597793,53273545946909576,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.274,1968-01-20,true,832154791.41,20:48:54.24 +-1629826157,82890888864932032,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.274,1967-08-09,false,211051050.74,21:32:57.50 +-382601456,84712972685864864,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.274,1960-10-04,false,874377081.965,13:29:32.46 +1603893994,11443956572267746,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.275,2006-08-08,true,131999884.261,18:11:35.32 +2103589071,21517055378153708,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.275,1993-07-14,true,698101999.293,15:56:48.38 +-1537450554,45369200466080768,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.275,1996-03-05,true,353850069.459,12:9:17.45 +897486860,41561586274422952,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.275,1976-04-14,true,193018262.649,12:35:21.5 +929576172,30879266649708208,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.275,1955-03-17,true,1262292207.22,12:3:46.45 +75069947,55141577115397872,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.275,1965-06-15,false,796094497.399,21:40:36.43 +-1557523104,85157164694916800,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.275,1990-06-24,true,1226100576.83,8:10:12.14 +-106558478,88871625525126640,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.276,1987-08-17,false,535220780.809,2:18:42.9 +-1169525287,5563832737489951,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.276,1979-03-20,true,832076073.041,11:20:1.45 +-602615395,88284514723555824,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.276,1954-11-23,true,256325191.016,10:18:17.16 +919968270,39105502663221688,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.276,1987-03-24,true,1255509168.53,4:34:12.13 +399362135,44565612588512184,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.276,1964-02-21,false,654603260.149,15:6:27.29 +-796788999,65715233019006416,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.276,1966-03-04,false,1271122686.72,9:18:50.42 +1710631647,46009049510504656,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.276,1996-09-11,true,924775296.849,13:33:44.14 +267629427,13481611525731052,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.276,1958-11-20,true,603156812.733,14:6:43.13 +-1231241924,54697253552071176,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.277,1965-04-13,false,583735606.106,6:56:55.31 +774080536,57867795683229312,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.277,1953-03-10,true,143675774.463,4:54:42.31 +1634440419,7576588294133678,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.277,1962-10-16,false,1081399747.61,21:14:52.8 +-952606193,63401559229438024,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.277,1983-01-03,true,371408843.706,6:32:57.16 +-906763085,91621972141711056,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.277,2004-09-22,true,158166855.657,3:48:19.5 +-1791683877,48139203170404176,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.277,2001-08-05,false,276762859.469,1:37:37.43 +1167554234,47501193208056064,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.278,1999-11-17,true,288540088.944,6:40:4.9 +-1654031548,86405417339327488,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.278,1957-07-15,false,443051044.203,18:6:42.28 +-259892049,58061512338140696,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.278,1970-06-25,true,402572713.57,9:11:34.39 +1260190050,87697037712901568,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.278,2007-07-13,true,1128335422.56,7:7:56.57 +-1131589373,1646808298439209,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.278,2002-03-07,false,1144584339.06,22:35:9.33 +-1583032070,26045087135779164,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.278,1967-02-04,true,335580455.32,9:57:39.12 +-584610744,70726723237254528,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.278,2004-06-17,false,1089634859.61,10:9:20.3 +1197036419,71805104960808192,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.278,1974-01-24,true,804569243.079,13:51:49.33 +-845227865,10411476327881554,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.279,2012-02-14,true,1273430206.2,10:32:34.13 +1039234548,46673048043385112,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.279,1978-10-20,true,3120746.01893,8:16:7.9 +149960154,75774522760983472,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.279,1959-02-27,true,1309390244.41,15:37:17.33 +-465561528,54112196681821112,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.279,1950-08-17,true,1423580002.67,18:50:20.27 +-437805292,85622370562716688,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.279,1963-11-04,false,642799084.701,20:56:28.5 +168201669,57967060205084928,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.279,1952-01-27,false,200511877.677,20:56:14.18 +492682193,76382797636418256,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.279,1994-09-17,true,62596204.254,16:56:25.25 +-1919607347,85058411161694592,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.280,1977-01-21,true,1270470974.78,12:58:22.17 +-1070510836,45967059969020632,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.280,1972-03-26,false,380212255.206,18:43:15.31 +-829074724,91111812436270704,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.280,2011-09-10,true,1106085886.86,15:46:37.15 +-588197252,55187040812560608,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.280,1986-06-18,true,243663223.306,21:11:37.26 +-552196394,40434031332436440,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.280,2012-11-05,true,123540351.589,1:14:54.38 +-1850474817,89380460924333440,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.280,1951-02-02,false,557428326.411,1:22:14.38 +143934839,91668820271871024,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.280,2005-07-16,true,844830697.009,19:15:36.46 +782097568,71762818089684136,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.281,2003-11-10,false,1222632626.54,11:47:18.3 +1887567321,76560008594149728,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.281,2011-05-07,false,348704602.043,8:25:21.48 +517931367,56779209171823648,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.281,1981-04-02,false,1333538373.87,8:28:36.27 +42380257,49040684058348712,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.281,1964-10-10,false,947469983.05,20:16:47.52 +969039797,37124466383897848,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.281,1971-11-11,false,1179520966.65,8:36:7.25 +-892014998,3126385168931502,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.281,1966-11-06,false,534030862.125,7:12:39.4 +-1285669422,54934411569038904,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.281,1995-08-12,false,14783147.1028,1:29:51.20 +-605004352,43420534181176000,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.282,1955-01-17,true,696991393.935,4:11:40.35 +-566173370,50253187190329408,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.282,1989-06-26,true,260873193.214,5:14:24.12 +-452832095,24150144278337352,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.282,1984-05-04,true,641815308.392,4:35:44.22 +-1984436853,29076620162109944,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.282,2008-01-09,true,1272875198.37,12:55:14.27 +-211940288,28378928250701744,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.282,1977-10-01,true,1218114069.8,13:51:19.50 +509146097,90451987986800720,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.282,1957-02-15,true,1186711245.55,8:16:31.34 +-1190082655,39367703290519440,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.282,1954-10-07,true,1321902832.22,11:51:18.19 +-762062889,79629359986459600,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.283,1992-07-01,false,1099573085.6,10:2:56.5 +1986060307,50074741737598008,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.283,1957-05-05,false,67847503.0412,13:57:27.15 +1105284709,21542589303496672,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.283,2012-01-10,false,1005729301.97,18:32:35.25 +218145395,11062616537454562,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.283,2006-07-21,false,1297139539.22,7:15:29.44 +-1560138002,18944605634084640,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.283,1985-11-09,true,1057340304.03,8:7:50.7 +714418503,87976566891654192,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.283,1952-06-08,false,751875373.523,21:49:39.49 +450730735,36404721359375128,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.283,1993-07-05,false,202703270.933,21:1:19.35 +2118450518,24236439473035952,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.284,1950-10-17,false,511026234.165,15:43:47.48 +550420506,1842569807059763,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.284,2007-02-03,true,1039097713.75,22:58:42.52 +-974440109,46769815595433896,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.284,1997-06-19,false,724758390.578,6:7:32.45 +1679105732,28888898977741680,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.284,1995-03-07,false,428149678.413,9:58:33.23 +-1693379220,61378011570571272,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.284,1968-10-16,true,924568173.434,8:22:41.27 +-1269640398,36039336917463536,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.284,1986-01-04,true,51896117.94,11:31:31.26 +-758654313,73428222803535264,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.284,1954-08-20,false,735748428.113,15:48:16.49 +-1838128971,7553282053284475,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.285,1976-02-03,false,575847273.622,18:48:44.26 +-472207641,17158648843677164,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.285,1981-04-10,true,1121713297.04,5:14:52.17 +-266663054,59407527737381808,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.285,2006-09-11,false,366459399.782,1:48:19.30 +1768921142,25882961087238052,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.285,1984-08-19,true,1334966522.95,9:23:43.8 +2061239370,37202118327858752,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.285,1962-06-26,true,262762615.788,20:31:47.27 +1980951281,10564327948866038,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.285,1970-11-14,false,40149169.5448,19:16:28.33 +-748392042,58820309012870680,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.285,1970-04-03,false,684974890.932,19:16:36.42 +573686123,43929305081220024,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.286,1952-04-23,true,1195610024.63,3:18:43.18 +-1391361251,70698832418085808,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.286,1951-11-08,true,132345765.35,19:28:13.15 +-894719893,33800985368038556,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.286,1990-05-14,true,1184607259.11,19:8:24.1 +-1739147093,55608487036181160,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.286,1997-11-21,true,893118791.511,7:50:2.16 +1015959574,87587051398966432,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.286,2000-10-26,true,598958645.118,5:45:4.21 +489397369,66406294945666184,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.286,2001-08-16,false,936398034.487,20:20:52.39 +-2045708492,42270019337702128,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.286,2004-03-09,true,789481691.783,14:58:5.7 +1879189703,16012936258897080,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.287,1952-05-20,false,377134572.191,1:47:56.53 +-107647081,84976823211438720,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.287,1964-04-13,false,247772610.96,20:12:58.41 +-1960242193,57501163566175664,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.287,1994-02-14,true,1433076003.69,22:46:38.41 +1726717432,6218197778919998,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.287,1993-07-26,true,1342050211.31,6:53:49.17 +1561650074,77524597300834176,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.287,1956-02-26,false,897868395.685,19:53:44.17 +1084307358,65236795033701376,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.287,1962-09-16,false,877574720.989,4:28:50.22 +-1456911150,48960084858719408,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.288,1987-07-02,true,1333228698.78,2:21:49.3 +-1407244581,2295226806574623,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.288,1976-06-15,false,742295575.368,10:44:15.35 +-406592325,25805793502687428,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.288,1987-07-13,false,799789609.832,9:28:33.41 +-12106070,54765300904561808,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.288,1970-04-18,true,1089883955.19,10:9:42.28 +1754286003,39978870921187880,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.288,1955-01-01,true,1243367794.47,18:39:15.7 +-1167974599,76765053211854256,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.288,1971-04-06,true,291086930.834,14:52:22.9 +603082546,59710860246747936,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.288,1964-05-26,true,318866503.025,18:20:4.58 +-837702610,54626540874338296,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.288,1997-07-25,false,1291413582.74,16:27:57.23 +865777555,33055232575813828,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.289,2007-10-21,true,505324155.151,22:33:27.41 +1094868112,6495701613438280,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.289,1991-11-08,true,764392531.473,19:34:56.33 +689896227,77296203076213072,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.289,1958-01-01,true,61395506.436,12:1:5.32 +1533676375,87164242410198384,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.289,1963-02-11,true,225922699.747,16:11:12.47 +-1431849144,88561678508891776,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.289,1998-06-15,false,984106020.085,1:12:49.22 +1061894830,78539809983919136,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.289,1997-01-23,false,107202042.666,5:33:2.30 +-2061178390,88720417069132224,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.289,1978-07-08,false,698356244.736,18:40:57.41 +1261328848,41120385540200112,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.289,1999-08-27,true,270169863.268,8:27:6.1 +879486646,69767546353405128,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.290,1975-09-01,true,516973907.7,20:37:45.33 +-365365171,20213180892426344,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.290,2001-01-26,false,1311878222.13,10:39:32.1 +1029520125,26816688466055836,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.290,1985-06-17,true,1198619151.44,16:35:55.4 +1930270995,8869214509515817,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.290,2000-10-10,false,701457886.106,8:33:30.28 +1769994515,76772259286243808,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.290,1958-02-07,false,1119147265.63,10:3:30.1 +-700659008,57799060343512072,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.290,1954-06-07,true,1227559647.24,5:29:38.51 +1956143243,72093132242334752,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.290,1993-02-07,false,237912942.362,8:48:51.36 +-1053217713,28088360047748044,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.291,1986-11-27,true,713352281.39,7:33:56.26 +1669429436,14741042308755354,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.291,1998-10-26,false,1131541470.45,18:25:27.2 +988940679,78465482026215680,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.291,1953-09-16,false,301793113.102,20:35:51.4 +2064526419,31744442180540572,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.291,2008-04-07,false,190029524.655,14:26:40.14 +597205143,38188073851761784,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.291,1982-10-23,true,1005672538.84,8:56:6.24 +-1840973482,76477554925580160,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.291,2013-10-20,false,89216701.3297,11:31:5.5 +-555271287,9764614475072410,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.292,2012-02-16,false,474091533.689,17:15:28.38 +-996359163,4247509552171571,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.292,1991-10-09,false,1010358582.2,9:4:9.20 +522939511,80861428842059552,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.292,1986-04-18,false,1218916040.81,20:41:26.33 +490966358,83278064847615472,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.292,1956-01-25,false,1149933421.33,21:11:1.29 +562552117,33489669850826188,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.292,1976-06-23,false,408778312.023,17:19:7.41 +1568712467,2709453083045151,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.292,1995-11-12,true,559101729.607,1:3:7.34 +146021482,84260066185301328,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.293,1995-03-11,true,240204376.45,14:46:10.4 +158394378,80613584055915344,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.293,2002-06-11,true,7808935.73621,7:31:5.40 +1540494507,34437269621788592,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.293,1998-02-06,false,830302980.521,9:33:9.4 +1821965673,84355342435094784,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.293,1982-02-18,true,496496903.038,9:25:42.39 +505766321,73213745945904736,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.293,1969-05-07,true,383662378.215,13:12:22.57 +-1393825767,6669328433685842,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.293,1960-06-19,false,831789455.125,1:9:13.51 +1237058491,43251968232288448,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.293,1968-04-15,true,1196407203.71,21:46:29.23 +1790278606,23746634948424192,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.294,1975-03-07,true,258566545.724,7:23:27.44 +943096337,79285905246099536,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.294,1972-01-02,false,626263732.06,7:42:19.47 +-1574046769,24269425955042132,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.294,1960-09-25,true,423868117.134,21:48:55.1 +-1105451241,42581022633894552,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.294,2002-05-26,false,723681114.785,10:20:19.51 +582809761,65521462623334104,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.294,1956-11-06,false,806988968.249,18:2:15.18 +-726530836,83757307055911744,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.294,1961-09-01,true,528240023.264,22:6:24.39 +1847028096,33052456163057092,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.294,1969-10-06,false,408662775.153,1:50:58.18 +1906633045,23658160001215060,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.295,1973-04-01,true,1196364549.93,17:11:19.56 +90103936,36561302759162872,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.295,1985-04-14,true,821706298.296,11:49:15.4 +-25393108,86162599424955440,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.295,1970-07-27,true,698773749.164,8:41:38.38 +1410017711,24322694363270196,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.295,1989-04-21,false,1424583590.07,2:19:43.58 +2114455685,71278486513307536,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.295,1989-05-22,true,388928457.56,13:31:1.49 +-1330622941,22702379534816708,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.295,1970-05-08,false,1289059169.67,6:45:31.10 +-2087698357,23326626730944696,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.295,1992-11-06,false,602003327.203,11:11:52.6 +756056213,86470354814011328,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.296,1951-10-25,false,961238286.786,19:13:44.32 +-343095714,32525079821766780,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.296,1981-11-15,false,1242723135.47,12:1:43.17 +-1839278231,56428660371968296,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.296,2013-09-17,true,158061478.502,5:55:57.33 +-1979794729,23537593983099168,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.296,2006-09-03,false,883231040.726,8:9:22.49 +-1891969594,62065546695598688,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.296,1959-02-17,true,1290100986.9,15:53:34.37 +-1704403995,60788281312073360,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.296,1974-08-04,false,1210603070.2,12:28:12.25 +-541492708,11399169896990690,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.296,1953-05-20,false,604891133.107,17:16:19.10 +-791970843,21110573246898504,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.297,1965-08-12,false,1034252295.09,21:10:26.6 +-1727388003,18820324376096584,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.297,2013-05-19,false,383701082.669,3:55:23.21 +-1697206256,53591506396210512,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.297,1976-08-19,true,294017665.797,6:15:55.34 +-942180537,81114530834132320,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.297,1962-02-07,false,209396109.523,12:1:39.40 +-382474824,13124817550319494,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.297,1984-08-06,false,962697603.666,18:47:1.20 +-475998110,38944594455391952,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.297,1983-10-05,false,1191023339.61,1:13:21.11 +7804224,41872648291699600,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.297,1987-03-14,true,820321398.427,11:14:18.26 +1672245462,63749784531833312,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.298,1951-01-22,false,375042052.391,8:6:26.15 +1296473095,33449426250529412,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.298,1966-05-25,false,1422409452.65,20:29:46.11 +-416561819,5272454256597228,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.298,1967-02-09,true,242574454.846,1:48:54.43 +594444565,18109282713260084,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.298,1993-08-02,true,1424378529.6,8:48:26.54 +1097838478,47026288965478912,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.298,1973-08-10,true,666466019.316,6:13:28.36 +1641413745,66818896929075016,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.298,1989-02-07,false,768102579.065,16:46:28.37 +1651924098,85911691447982896,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.298,1980-10-05,false,937641684.508,16:14:28.44 +651472878,85501650497501488,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.298,1972-10-02,true,13189352.9672,4:11:54.1 +-930778093,31903643946278492,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.299,1991-08-17,false,1373052708.8,7:48:54.23 +-1933982352,50088947420098032,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.299,1974-11-03,true,1020340236.03,16:40:9.29 +-874729106,21034445649398068,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.299,2005-11-07,false,266332356.468,5:12:24.46 +-1447279872,13597274366777376,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.299,1994-03-03,true,1271494793.65,12:41:58.44 +687447333,64996036491097344,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.299,2012-07-11,true,1346122030.78,10:33:45.2 +1613433789,16663616994369056,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.299,1992-11-03,true,915383680.141,1:36:15.30 +-752098972,20868166236228240,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.299,1980-02-22,true,1305794611.39,18:46:20.46 +-1215438925,76553716701895760,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.300,1989-01-23,true,806186933.393,15:36:37.19 +1297432570,78448932335519184,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.300,1952-01-14,true,514288981.748,7:23:28.30 +1346696429,87953319815739328,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.300,1984-03-06,false,462134124.681,12:56:56.15 +993332362,46812553331308496,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.300,2007-07-21,false,1225393846.22,19:41:30.41 +214082456,9419410992184710,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.300,1979-05-27,true,1122239360.26,9:9:9.25 +-562003041,73596035327546160,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.300,1960-08-20,true,1120368672.7,17:41:9.4 +1795744779,68143288410431360,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.300,1973-10-02,false,235372387.248,1:36:54.48 +-441699499,40006460973844832,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.301,1989-11-03,false,330454809.669,6:13:11.50 +564337416,10890746020803328,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.301,1964-04-13,true,298779706.837,11:20:18.38 +-892164885,36288403608513200,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.301,1969-11-13,false,898485669.51,7:40:47.10 +-1118361512,56505589134678000,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.301,2008-08-02,true,1374381687.36,5:15:49.9 +-241717465,21500883634218384,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.301,1956-06-21,true,535473911.149,15:8:34.47 +-1381493045,12745758821996238,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.301,1982-02-02,true,128490600.451,1:30:33.41 +-2082533227,75288777741643488,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.301,2002-02-09,true,495704663.242,8:57:10.31 +1097207392,18175189872992952,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.302,2010-09-07,true,263888721.78,18:40:18.42 +-1005871373,91198588822148160,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.302,2013-01-21,true,1386618660.73,15:42:29.46 +-960754437,30870339794831168,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.302,2006-04-11,true,1352310819.63,7:19:40.11 +1474282702,83873790916871136,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.302,2008-11-21,false,1355007801.34,21:24:8.56 +523422320,79848859281594912,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.302,1999-04-23,false,712991770.612,3:49:54.8 +-1179274921,44940416427170552,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.302,1982-07-16,true,30218240.4155,17:4:8.29 +1577387328,24140271765417904,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.302,1997-05-14,true,962951122.278,17:24:20.9 +363802008,49770924244054088,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.303,1988-08-13,true,1310160269.98,2:10:45.21 +-956451603,41247308898582016,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.303,1984-07-12,false,923043129.968,13:19:49.13 +296228454,81524806639955184,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.303,2004-07-20,true,327512370.23,16:42:56.7 +2128023524,58070438183263304,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.303,1976-08-08,true,319036034.383,14:9:29.5 +-1200065697,10152718778773822,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.303,1951-06-08,true,271546585.712,12:8:17.23 +1212279889,77023205644632256,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.303,1991-07-23,true,496038579.219,22:9:43.35 +-43426191,3165614324991078,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.303,1979-04-11,true,277455206.666,22:40:44.20 +-1376922915,4434902959481887,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.304,1986-03-01,false,1181000103.94,19:36:19.22 +342750870,82872909523107328,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.304,1984-08-12,true,1416669932.92,16:29:4.2 +-1064716017,75741775887888064,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.304,1953-07-11,false,1295627603.45,18:21:43.14 +344947271,22095416810533344,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.304,1951-01-13,false,247499226.672,1:57:42.50 +-196151333,66015556660935848,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.304,2004-08-10,true,908669992.322,15:18:41.2 +-563680036,27214609567443168,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.304,1997-09-22,true,65864865.4806,9:55:58.8 +578489403,71544503811327928,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.304,1958-02-20,true,421500315.564,19:43:39.14 +-1544946584,76768967756162304,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.304,1958-04-23,false,911421929.289,12:14:53.4 +1559670487,54228125504051776,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.305,1988-10-16,true,559937078.283,4:45:14.7 +1567855541,38097861819904192,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.305,1955-02-06,true,353410571.85,22:23:55.47 +-1978833826,91856238028732496,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.305,2012-07-06,false,508099776.963,2:15:20.5 +-1066884946,67747152932994240,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.305,1990-08-10,true,388261997.323,6:10:6.48 +1707837254,34688700589797376,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.305,1976-02-21,true,901480884.759,8:43:34.6 +-6739804,2871886401575362,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.305,1988-07-04,false,991659618.728,12:20:58.2 +177040515,90530519170178640,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.306,1994-03-07,false,1228752751.51,8:49:22.37 +-1940781150,14733066876983060,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.306,2001-06-26,false,864649820.407,4:37:53.25 +153097150,15882242544039874,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.306,1984-06-25,false,899923442.99,4:35:47.54 +1579329900,76469430918596432,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.306,1966-06-26,false,1419466327.1,17:43:4.43 +419958754,33423063678744852,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.306,2007-05-16,false,871991887.803,2:11:53.39 +-1111915825,69266578213264192,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.306,1954-01-24,true,294745844.775,15:37:5.54 +991599578,2082515225216030,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.306,1958-05-04,false,492952427.699,20:42:51.58 +1384002695,64232741621209080,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.306,1964-03-21,false,1110177748.82,7:40:34.52 +-257260351,30204543489259828,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.307,2012-07-11,false,511230801.73,10:19:29.30 +1014523502,30429166063212800,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.307,1990-01-17,true,1423981632.37,16:4:48.1 +1944571290,38672897584045792,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.307,1978-11-02,false,533965493.243,7:29:4.7 +1418141056,5705917268209592,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.307,1981-02-25,true,1009358564.01,16:1:55.10 +371675376,10235805153536462,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.307,2005-02-21,false,328259908.512,20:13:42.2 +1930172618,96103000499722,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.307,1950-08-02,false,700178692.768,10:7:7.50 +761963974,3131510276174715,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.307,1980-05-01,false,887885828.76,18:33:17.7 +-1385843744,422671190044047,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.307,1958-07-24,false,724060047.623,5:15:12.26 +1282929870,22229006958982544,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.308,1994-03-25,false,820847401.719,13:31:54.26 +1731053632,47490339691109888,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.308,1983-01-18,false,93593821.5529,12:30:25.7 +1139828469,91001760139836464,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.308,2004-05-16,true,198870345.122,19:58:33.40 +-776149738,27077517818010992,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.308,1985-11-01,false,1024243055.57,2:34:1.16 +880525297,56648661508631328,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.308,1985-10-10,true,835092284.913,2:39:33.47 +-313750981,90997890089286528,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.308,1976-07-03,true,33876784.2348,12:17:49.58 +-2112647416,52225166702747064,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.308,1980-10-26,true,752018194.39,9:54:6.5 +-1269242649,45720452894918736,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.309,1986-03-07,false,455628846.42,16:41:13.48 +-727108246,10040357007260784,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.309,1957-06-08,false,1372477738.74,16:47:49.24 +1562032328,47059489606185648,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.309,1984-01-15,true,1249387794.27,14:3:7.9 +-716112314,66839749441228840,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.309,1996-10-22,false,1380414446.52,14:7:14.36 +-1346315239,86805240507418592,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.309,1971-02-13,true,1251465059.72,13:30:1.16 +-1132943652,21025485912675604,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.309,1952-03-13,true,807704129.603,20:13:44.57 +-397458344,91349563713290256,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.309,1985-06-12,true,1252483026.78,18:29:20.43 +721296946,67205820398987192,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.310,1999-04-11,true,331315339.374,21:45:46.29 +-2087650531,22005108717712436,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.310,1950-08-11,false,1078074681.0,7:20:3.44 +910420784,15561427453898936,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.310,1985-08-27,false,1312900945.13,5:34:45.21 +-2126877721,39897482988402352,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.310,2000-03-16,false,1087531648.18,19:48:18.26 +-2089856523,86483053220420048,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.310,1952-07-14,false,733223019.065,12:36:15.22 +-1630165872,85435262184685040,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.310,1957-05-05,true,13293592.9317,22:26:30.20 +1493362487,68262331646595504,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.310,1955-04-09,true,221162729.442,2:34:3.7 +2014136493,61778968279574024,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.310,1971-05-22,false,503592678.645,19:20:6.33 +-1841414734,48224680770385648,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.311,1955-06-02,false,1382210873.51,11:22:56.6 +-1980027120,21677262826590804,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.311,2005-07-15,false,1406552565.7,5:9:14.22 +-916025668,15450816716897864,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.311,1969-04-24,true,542223875.357,17:23:31.34 +420117837,1881955658222756,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.311,1975-11-12,true,1391377392.72,18:31:22.1 +594783165,85056196396871104,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.311,1981-02-16,true,806831649.417,9:3:13.20 +-1785796830,65017716206075600,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.311,1970-08-21,false,1031246133.17,9:54:38.38 +1607276701,485107193450137,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.311,2000-06-24,false,837715904.034,19:2:48.13 +885208108,20701482983993016,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.312,1969-03-25,true,464925403.242,7:31:16.16 +-317667064,3562459732803246,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.312,1993-04-04,false,1210901628.87,10:7:44.46 +-1333620507,23650437222268960,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.312,1994-01-24,true,434997825.152,6:1:56.54 +-951532042,4483797477741445,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.312,1993-06-03,false,1283102003.09,19:23:18.36 +-1570397836,38200558485210912,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.312,1958-07-20,false,250948339.805,13:24:50.46 +-1203402644,58152494376135928,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.312,1968-02-08,false,161231462.392,21:29:14.8 +-2066545242,64117096278091112,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.312,1976-02-08,false,328023050.436,21:7:45.15 +881313457,28973483214287280,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.313,2012-01-13,false,639767430.152,18:40:48.29 +-1782654946,81482686997412512,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.313,1984-08-18,false,821763014.271,1:53:38.36 +-1874476950,12503200387328542,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.313,1959-04-19,false,1108576572.95,20:31:39.56 +-154508131,47912670577554688,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.313,1986-08-24,false,341482919.674,5:52:10.57 +240035821,47263853747034832,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.313,1971-05-12,false,710786784.609,9:38:1.53 +420287632,16694639374904894,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.313,2003-05-14,true,799496402.129,5:6:40.52 +-1105437294,43351223805789320,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.313,2005-11-26,false,1297881122.56,3:41:42.46 +-996417942,43692525860708536,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.313,1988-10-25,true,121503751.071,12:9:25.30 +1601744038,53559759285552416,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.314,2010-01-12,false,542294909.584,1:23:21.43 +-219852006,92164563546273856,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.314,1984-03-09,true,821682558.531,3:12:36.43 +-1595588167,13050614687365386,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.314,1970-09-03,true,1303149482.37,8:53:20.11 +1653336529,73145799695024208,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.314,2000-06-14,false,440008738.623,14:24:16.39 +-1702677656,41099936137630392,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.314,2013-08-23,false,122484766.874,3:4:48.32 +-1143388399,18614447474469664,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.314,2001-04-27,true,1119575835.97,6:44:25.17 +-1641355333,6558053454277550,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.314,1968-10-18,false,1291769373.57,9:43:57.21 +-367125051,73181941683389664,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.314,1960-08-07,true,177121348.24,6:15:34.27 +-705122287,88204262719718704,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.315,1986-08-27,true,1026019642.22,13:53:40.23 +1316593763,18327640837714208,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.315,2009-11-04,false,1063912921.26,8:57:5.45 +-1232054384,75058685753436432,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.315,1970-10-03,false,428050804.5,16:16:15.43 +1492558003,62218437478810904,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.315,1957-05-18,true,34623044.3513,6:35:50.34 +523307300,54553749419405384,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.315,1981-08-13,true,693000934.111,7:38:21.7 +-74506497,38432609462084944,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.315,1975-05-14,false,911311113.971,8:48:14.17 +-747820743,25062670702586440,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.315,1975-11-01,true,594514569.843,6:26:23.39 +-1442804187,80802877574841216,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.315,1967-01-20,true,663619251.826,21:55:21.23 +2054951193,43208604315886064,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.316,1963-06-05,true,62681559.9342,2:36:17.21 +1826145554,66643967059690464,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.316,1950-05-01,true,35468360.6716,19:29:14.27 +-1335901832,89429473298492144,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.316,2005-10-04,false,1268356991.9,1:43:24.27 +-254498706,31936776313744888,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.316,2004-09-14,true,1310053375.29,22:42:22.38 +-2059664405,43244301462722232,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.316,2005-04-18,false,35224444.215,7:20:49.45 +-1775404551,64270261882464664,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.316,1959-11-17,true,16673377.8098,10:34:58.56 +-821841260,62506290251120816,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.316,1951-10-12,false,892850936.729,1:50:39.35 +-876049252,37221799850881576,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.316,2012-07-09,false,745214043.406,17:32:12.37 +-638644973,57396366659252744,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.317,1999-04-06,false,681412294.213,3:47:40.25 +-1414084445,9233810549757666,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.317,1971-01-08,true,726228312.972,18:26:34.56 +785026501,58156750730798768,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.317,1955-06-18,true,273317621.453,11:43:27.56 +863960930,3713437319138611,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.317,1953-02-27,false,1063349296.26,7:43:27.48 +-284850600,88347341555141664,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.317,2001-02-10,false,1310812919.19,18:58:38.40 +1235837681,32473192498812764,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.317,1966-03-18,false,1407977491.39,15:52:1.55 +-49602201,17492334389792092,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.317,1964-05-25,true,44151218.1313,6:42:17.9 +-848358283,8171428735705651,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.317,1953-11-25,false,540332652.337,4:34:3.30 +628299676,49407799897389344,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.318,1993-04-24,false,235228265.421,10:36:56.11 +-679964714,52752757841309792,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.318,1979-11-16,true,1227093766.92,6:15:2.51 +677715990,67870473871830488,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.318,1961-06-21,false,424711673.495,7:21:8.58 +925338577,36710732440941608,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.318,1958-05-17,true,490912028.316,3:27:38.31 +-2012118929,83668121375957952,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.318,1955-10-16,true,995561125.668,13:3:43.3 +-1441288205,22626763130857852,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.318,1958-08-25,true,693669051.386,21:46:27.11 +-1985095412,77917720913799088,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.318,1974-11-06,true,386057393.161,6:15:36.41 +-949836040,40802390554209376,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.318,1968-05-08,true,1119616659.48,6:38:15.14 +-1433149649,73204480502790048,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.319,2007-03-02,false,266202271.495,10:29:7.8 +-109110162,11789502751216016,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.319,1973-10-02,false,785421602.868,3:26:40.3 +1563983109,35309128275805460,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.319,1982-08-10,true,118569680.527,16:11:35.29 +-1508134466,64243618519642360,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.319,1952-04-25,true,1426526592.35,13:5:6.17 +1218398586,5206793800503777,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.319,2007-10-25,false,547947084.839,2:50:26.41 +1051305648,64318203109114432,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.319,2008-04-09,true,239902722.382,13:33:23.27 +1925193812,82258853652016448,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.319,1974-08-25,true,766517439.504,21:28:9.33 +-422822433,61647191968127952,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.319,1978-03-06,false,498179101.634,21:48:24.21 +1796182205,90647805011145088,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.320,1981-01-25,false,518344711.64,1:2:7.48 +-1867245097,882791083827425,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.320,1965-09-17,false,599478687.375,9:49:8.26 +2091726283,88235270417698288,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.320,1991-09-05,true,51033841.9104,3:10:56.9 +-1227984488,90360874588097456,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.320,1965-06-06,false,647477510.037,5:47:33.39 +-2026148188,54987299008738952,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.320,1960-10-11,false,1368605423.15,15:4:7.40 +222756655,48081217761051504,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.320,1997-08-12,true,96894367.7236,21:48:5.38 +687439766,55886622328500632,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.320,1965-02-09,false,419809459.09,15:4:39.21 +-2040441340,45007532320007416,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.321,1953-01-11,true,1117066758.5,10:25:58.15 +-759941981,74431718919252304,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.321,1980-02-15,false,493680562.789,1:52:52.34 +-1917092530,52267311377148528,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.321,1969-10-19,false,966238096.545,14:46:25.55 +378857965,47762385732640656,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.321,1957-08-21,false,1077737350.08,18:47:32.50 +-784239703,27820993175570956,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.321,1965-05-20,false,451588290.207,19:19:51.25 +1770955607,76080117684367472,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.321,2004-10-02,false,225762941.761,13:38:45.27 +-462741504,3452674413007094,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.321,2011-06-06,true,1226219679.0,22:50:2.28 +2014948439,66088736236718920,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.321,2012-01-19,true,1021408891.74,15:18:15.43 +107164571,58166129255415632,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.322,2007-05-05,false,232126310.461,11:37:40.37 +-123029713,39984264089655464,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.322,1999-04-06,true,256957681.327,12:33:9.49 +-1227277776,70739116543157632,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.322,1996-10-16,false,948080791.227,15:37:43.11 +1860066647,64278542052794192,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.322,1987-07-22,false,1192213653.05,5:8:25.32 +-195934939,83438034486708240,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.322,1986-07-22,false,954160203.305,15:29:40.7 +1591038963,86687274721171920,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.322,1965-09-02,true,100898850.213,13:16:49.6 +-456795453,85830229178673952,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.323,2009-09-20,true,961692320.81,18:3:58.55 +-1780485601,27773518378671136,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.323,1950-10-17,true,635372152.376,3:8:44.12 +352311680,45809269997363752,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.323,2012-06-13,false,1349755576.02,1:50:34.11 +833283007,79563421865126560,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.323,1992-02-12,false,895324865.37,8:41:35.40 +1298568922,52205005587616072,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.323,1977-02-20,false,1297901732.71,3:12:56.10 +1067464436,503965179022581,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.323,1980-01-23,false,306555501.756,15:56:51.37 +-166939460,5292684790769582,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.323,1985-11-27,true,969998378.621,22:41:10.4 +1037265294,77274047836508016,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.323,1977-07-09,false,810692525.757,16:3:35.32 +1621275246,87636302977161040,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.324,1983-07-01,false,1047020394.81,10:58:11.10 +1166540546,66849173626251272,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.324,1977-08-22,true,663337327.265,13:54:21.22 +-1387840521,3813527938099333,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.324,1962-09-03,false,667895677.333,18:18:7.50 +-1425543498,13430458416758764,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.324,1967-10-06,true,852736784.736,14:34:50.37 +997779705,49120091263171216,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.324,1963-03-14,true,668548133.734,2:52:22.53 +-1148457211,77007608779336048,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.324,1967-02-27,false,1080148263.63,12:6:45.25 +-963294837,7871840308512164,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.324,1969-03-20,true,243358320.327,6:22:3.22 +-660180683,69717374167242304,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.325,2012-01-23,false,580035436.487,6:21:17.25 +761124497,12077392707527896,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.325,1951-02-22,false,729272110.581,5:20:25.21 +120604367,100535041615677,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.325,2009-01-07,false,411935923.528,8:36:45.36 +-1071690398,38433852321905672,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.325,1968-01-18,true,279143906.485,4:25:19.41 +817360473,9567158737230776,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.325,1976-11-13,false,321326540.623,14:16:17.16 +1476255837,41246410473883536,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.325,1962-02-14,false,1227374466.62,3:46:30.50 +-2104489191,21503634821734556,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.325,1995-11-07,false,876517447.89,20:1:7.30 +210437190,41103494566157168,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.326,1976-02-22,false,686682706.427,11:2:7.36 +1906946506,63112635575595920,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.326,1953-05-14,false,1295414030.79,18:7:57.28 +-2035234303,70331861456385800,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.326,1960-08-25,true,967425868.966,2:22:46.44 +-1352903572,85311346349704384,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.326,1978-02-02,false,42881271.8661,1:8:49.56 +-1538376678,82496647518216016,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.326,1964-10-21,true,929182135.695,2:18:6.9 +1723518421,75225240251516544,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.326,2008-03-22,true,265520234.99,2:33:57.58 +1252699840,91904531374562368,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.326,1979-02-03,true,28516680.2159,20:14:14.19 +-364672760,60204551247658336,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.327,1975-09-04,false,87622881.3358,18:15:18.5 +-760035032,31385723118207416,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.327,1960-06-24,true,998132322.798,4:44:34.23 +-848437791,89036839234231584,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.327,1957-05-05,false,1365645745.78,12:25:26.45 +333607645,84206884843991552,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.327,1962-10-27,true,753418418.873,21:42:48.6 +1472273687,85422658476438064,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.327,2010-06-06,false,1021137269.88,21:43:53.16 +143637504,63210009408226600,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.327,1957-09-10,true,1177497534.39,11:12:14.36 +433433306,36582466369606832,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.327,1969-03-04,false,1327250699.21,20:24:21.56 +-621271494,5635079029893479,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.328,1951-09-22,false,963873901.935,20:49:18.19 +238401057,61407902013689264,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.328,1974-03-01,false,1318767307.02,19:16:30.51 +-1116443277,26532351822741504,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.328,1986-04-08,true,996177330.235,5:27:6.3 +989744401,20420990432928716,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.328,2006-07-26,true,963889612.8,16:34:1.27 +1732185411,17535816421074104,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.328,1965-11-19,false,1166296494.1,8:48:7.54 +446105301,50300167161291064,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.328,1992-03-12,false,1000070107.92,16:28:53.4 +-1768812861,32665067198935152,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.328,2002-05-03,false,1177984123.84,5:18:34.50 +-178157819,18154725524303608,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.328,1998-07-03,false,908048342.303,13:41:14.39 +2069948973,89970772562641536,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.329,1967-01-15,false,517832870.305,3:7:13.49 +653748229,87543327175819504,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.329,1997-03-07,false,628752085.064,20:25:3.2 +-319724258,43479636890225920,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.329,1961-01-07,false,545782210.305,9:52:7.6 +-1198703296,56813166883463488,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.329,1962-07-21,true,583235647.352,20:28:35.54 +407055556,56128898860608472,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.329,1987-11-27,false,668679527.097,15:5:46.9 +-194113101,35772476181889168,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.329,1963-01-09,false,74207853.0138,15:25:25.12 +310366397,29133527409825628,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.329,1984-11-11,false,192806688.199,5:18:7.51 +1496571096,62900649269527376,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.330,1964-06-01,true,27410129.8181,20:58:12.27 +1740652802,72737023460241664,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.330,1994-08-03,false,926930077.082,15:32:15.4 +2026604112,81983695404043952,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.330,1970-07-22,false,860353803.461,12:23:56.12 +-741993678,43693957671168736,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.330,1982-08-22,true,1430467739.86,17:41:17.9 +1125608715,41129848984706120,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.330,1998-08-17,true,1394260412.71,10:7:6.58 +1734904651,18653782200500376,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.330,1993-11-06,false,1399260529.91,22:46:34.23 +-2065706502,25253913494270484,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.330,2011-09-19,false,1115218961.22,13:8:18.15 +1751334950,63194273629949872,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.331,1989-01-18,false,665706171.164,19:31:40.3 +-2000421424,18326693732152268,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.331,1994-11-20,true,649433807.355,16:29:17.15 +-1213806174,88233613310022448,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.331,1961-06-07,true,1346915566.11,15:49:15.12 +-1876118055,21773948134944144,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.331,1975-06-26,true,1256051777.7,15:25:44.30 +-734207390,89119435593716480,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.331,1953-05-04,true,547913016.149,7:47:6.24 +1045994370,54299819978748656,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.331,1992-03-03,false,319670481.072,19:3:29.39 +-2027001,88718394280898768,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.331,1971-01-02,true,148020805.343,6:39:21.56 +-1772550366,15243262653345424,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.332,1959-10-05,false,121701289.767,10:6:14.44 +446225487,39410275617020704,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.332,1960-03-08,true,1046976040.74,3:11:21.51 +-271687219,86680833916373152,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.332,1954-09-09,true,1098198770.33,9:18:33.58 +-1608345529,48450150756902368,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.332,2012-09-05,false,85879814.4231,12:7:41.46 +-1620621352,82184581992135104,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.332,2013-03-25,true,11654924.0108,9:23:52.15 +651134949,28836321541265328,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.332,1998-10-13,false,1335781461.27,20:15:23.17 +1987865793,33836026395143528,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.332,1982-06-15,true,1066344457.45,13:50:25.35 +1644045492,53001232835361464,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.333,1979-03-17,true,640689490.32,13:56:43.1 +-367573894,62577958027892912,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.333,1970-05-09,false,524797887.369,11:23:38.7 +-1779266099,77303216082464272,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.333,2003-07-16,true,1011798932.25,21:7:30.20 +-189709213,57226163136426056,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.333,2008-08-18,true,386990207.621,18:19:14.11 +1599832979,58924562323508328,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.333,1957-08-01,true,1337799382.11,21:56:40.37 +-233918850,75633328692242784,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.333,1953-04-27,true,1355736716.19,8:22:35.54 +-797904396,42826434260886480,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.333,1957-02-06,false,1317303094.64,4:35:54.25 +-1880518546,40906302828726768,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.334,2007-03-27,true,426490889.628,16:3:1.45 +-1265611890,58156949944663192,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.334,1956-07-21,true,798172538.696,4:31:17.35 +598806312,87062219538940208,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.334,1961-10-27,false,1314539959.86,11:6:53.12 +1683045480,56106523216059520,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.334,1964-09-23,false,311901342.651,18:26:29.18 +-119769090,25118950936284244,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.334,2003-06-27,true,947575955.159,14:33:51.51 +1267274228,38952286908110448,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.334,1960-07-26,true,1423842303.56,5:27:29.23 +877907197,66794741106328176,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.334,1965-05-22,false,492180351.769,5:4:27.6 +1946875728,71031829400760288,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.334,1970-09-06,true,263663538.759,2:15:57.49 +1610461617,23133358836932444,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.335,1972-02-19,false,1273053407.34,21:23:2.4 +-1333423564,16841046701181860,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.335,1989-09-04,false,509777505.706,11:47:7.36 +-1732640984,73683643319768128,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.335,1997-04-15,true,1039603353.13,6:35:47.28 +1444139823,42458626237934672,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.335,1965-08-25,true,196928181.627,2:47:3.6 +26982134,45788437239987952,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.335,1975-09-21,true,122962814.086,15:2:20.17 +-571777752,85720498581060688,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.335,1986-07-27,false,732562227.293,13:26:6.31 +-1905734437,16131270845116898,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.335,1972-10-18,false,1379895370.1,7:27:19.57 +1780277903,22837923758100080,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.336,1957-02-24,true,518075384.156,14:10:38.39 +-1675715584,87739178688235648,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.336,1967-09-13,true,1224294636.84,16:26:38.4 +1870562998,85810653673324400,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.336,1998-08-02,true,1140649847.85,17:28:6.29 +1703965102,66800774809553848,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.336,1955-04-17,true,415846096.149,12:26:49.57 +-1708975247,36112948841533608,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.336,2003-01-04,true,106545897.684,9:2:46.55 +-1241829571,12310304828030086,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.336,1972-08-24,true,198399583.91,10:12:11.57 +-1472015817,49505461648441256,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.336,1955-09-23,true,931867927.842,7:48:47.40 +328770773,32663639719505900,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.337,2002-07-23,false,804885499.152,16:57:39.43 +-872288131,54275359700297128,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.337,2011-01-02,false,629144565.15,3:32:32.16 +1385566488,2860958639802726,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.337,1966-05-02,true,432246078.196,15:17:17.7 +900820157,55863470986844896,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.337,1971-10-11,true,1356223866.04,22:21:18.39 +690507641,54068747314237536,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.337,1987-09-27,false,647726286.705,9:36:8.48 +878839789,1159845328561315,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.337,1975-03-20,true,1367697072.69,18:30:39.31 +-1165134768,64885771574827912,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.337,1951-05-06,true,443065989.928,10:11:50.36 +-557174690,11306334862493390,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.338,1958-01-14,true,491188966.39,7:51:23.34 +-1546661461,86736162419016640,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.338,1973-07-16,true,980983003.48,11:5:1.30 +1837603950,61875493845578632,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.338,1967-01-08,false,1001062619.06,10:57:36.48 +1968490798,10631234645554166,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.338,1950-10-21,false,653123648.456,2:13:34.43 +1450024300,18232623036025896,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.338,1998-08-14,true,256140657.582,19:7:54.25 +-2024268521,23918540339231232,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.338,2005-04-13,false,734760855.535,16:11:57.28 +284033593,36854854431754448,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.338,2000-07-26,false,665367839.383,8:34:49.1 +1548379658,11227060262227928,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.338,1952-01-26,true,949150947.747,2:5:56.51 +1943147680,32322334591326876,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.339,1985-07-18,false,422046391.256,9:8:21.40 +47521976,68684254510122376,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.339,1957-01-24,false,1335805325.83,16:55:40.5 +912737969,9977203181919344,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.339,1974-10-02,false,734647342.049,2:55:36.25 +1150158865,16900023050217144,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.339,1985-03-21,true,1112903166.12,9:5:39.38 +491900364,88229852533173376,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.339,1988-10-01,false,1380306084.8,15:39:28.8 +367492788,31658025451115932,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.339,1989-09-08,true,950720603.456,5:30:51.22 +-587822192,70274731846346120,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.339,1956-02-22,false,424386358.025,13:53:52.37 +1861535367,53879090373147752,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.340,1993-01-16,true,489690047.093,14:28:49.38 +-1568673373,37369066213893304,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.340,1958-04-18,true,70465196.01,2:5:58.13 +1329154543,90691831760986560,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.340,1952-09-12,true,683710989.764,10:22:54.3 +-1881971695,42236722029322488,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.340,1993-03-03,true,822996490.582,6:21:21.13 +-1926313885,41729951427593552,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.340,1978-06-19,false,1167302851.26,14:42:35.56 +619865810,57983462837899184,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.340,1997-02-27,false,894237949.574,22:15:14.35 +-1003475083,13740113719515586,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.340,1970-05-27,true,389232933.427,7:15:57.24 +-625038525,33036293339468124,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.341,1952-01-10,true,321867198.16,15:35:5.5 +587002121,28490899538467276,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.341,1957-06-13,false,742403006.481,3:50:7.29 +1880633642,78713440756270672,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.341,2009-10-03,true,39623683.2843,7:55:2.52 +-1223742342,37661537693756552,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.341,2010-11-22,false,634244513.529,14:50:26.23 +296491517,46671843822722704,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.341,1951-04-06,false,1095754739.97,8:24:1.36 +2065930292,3542798030120888,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.341,1956-08-15,true,1196900559.41,3:3:46.31 +-407526171,33855505770521824,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.341,1989-09-14,true,160436414.719,19:34:3.39 +-823675509,57251961199646656,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.342,2002-08-26,true,987098265.766,10:10:13.53 +-1626811031,69291969281756472,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.342,2004-05-05,false,1112872117.61,17:16:56.22 +-1650855354,16227550422789058,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.342,1955-04-02,true,565134346.89,5:52:30.44 +-2103649441,56015126766683432,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.342,1996-10-20,false,621228332.534,6:15:18.36 +-906128325,32418035713002252,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.342,1973-02-25,false,702224152.781,14:25:23.32 +2004176583,80797278218618304,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.342,1968-07-11,true,1349101723.96,12:12:26.40 +-1025553549,33848206023603404,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.342,1950-10-26,true,894189802.971,19:9:39.40 +-1441875701,77268881035935568,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.343,1984-01-14,false,837881293.155,16:19:30.13 +1005538245,65926028183509432,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.343,1960-01-04,true,622312341.712,22:35:49.43 +-1945076200,68640844740367496,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.343,1995-05-15,false,1261213960.89,18:21:57.8 +-1978804847,83311166527065120,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.343,2013-02-02,false,626981996.004,13:27:38.21 +1962666863,26006314717169500,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.343,2006-09-25,true,1055102511.11,3:57:3.42 +589094458,64054640580812216,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.343,1961-07-12,false,158642667.51,9:40:57.53 +-602487517,38307956200466176,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.343,1962-07-02,true,956044380.371,13:38:27.12 +1936624172,28593193158967892,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.344,2013-09-25,true,1316461356.6,20:14:54.18 +279274474,54856437333432328,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.344,1969-01-19,true,142816952.74,18:9:57.42 +-659696390,91447345084701200,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.344,2000-02-08,false,1400271054.72,11:21:38.49 +-500137369,19808531837630384,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.344,1965-01-24,true,1024650061.91,22:21:12.1 +1248916869,73091137719811600,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.344,2000-01-09,true,823680404.903,22:57:8.38 +-834788515,917888284122951,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.344,1991-06-19,true,209890993.524,4:22:10.16 +1366469396,52674910659827008,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.344,2013-10-25,true,91446990.8775,3:55:37.33 +1940964356,8911225937412506,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.344,1992-10-04,true,403489617.597,12:48:23.47 +-342117652,22187101927725752,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.345,2001-05-18,true,127338955.344,7:6:32.51 +549426954,77085147176902480,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.345,2007-02-06,false,1216776158.42,10:40:43.27 +1223543938,82045798676038480,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.345,1969-03-24,false,96906194.0413,13:14:7.15 +-268280669,65575134921804640,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.345,1976-06-16,false,318518638.784,15:43:49.27 +-1363342153,78433062663738784,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.345,1980-08-20,true,822870647.813,8:9:24.48 +-435749726,41256271875279416,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.345,1962-07-06,true,324075263.466,17:6:22.41 +-1390543879,86359897190586944,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.345,1969-08-15,false,298690143.555,17:14:10.33 +622307259,56667382947358656,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.346,1967-05-14,false,804003137.927,4:53:4.44 +-624255707,18202408299751476,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.346,1967-01-02,false,1247253770.67,17:5:44.38 +-1575123785,58216057385896560,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.346,1961-07-07,false,223846781.819,22:16:54.22 +-1211387061,45065826191144840,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.346,1991-07-17,true,1005377805.7,3:38:27.57 +-360563764,25687873387930276,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.346,1982-11-22,false,1035930666.98,13:46:18.50 +-1306705280,46685331172202024,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.346,1999-03-05,true,1239384601.63,20:54:10.38 +1880133888,60437098382363176,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.346,1970-08-25,false,576558753.325,20:16:28.43 +676553549,33197615005134284,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.347,1954-06-11,true,1140705133.27,21:35:24.54 +-560166024,74571737640303792,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.347,1981-08-25,true,153718367.069,5:14:3.5 +-1647025304,10975476357584906,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.347,2007-08-01,true,716302937.95,20:10:17.54 +851894311,71802965140442552,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.347,1967-09-10,false,1065831261.73,4:49:4.12 +-1076725425,87807612592716272,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.347,1961-06-21,true,1203049073.62,21:15:2.14 +-1429811449,663435453792542,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.347,2009-05-24,false,1174994675.21,11:21:50.29 +636736228,76364050807743600,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.347,1994-06-13,true,727982274.387,18:46:31.9 +-2022986247,11492429902363516,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.348,1992-08-22,false,398239437.65,21:50:27.9 +-368976490,15509505477445058,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.348,1990-05-21,false,1132709812.68,16:51:14.31 +836099933,56125399424662960,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.348,1951-11-09,true,629365410.887,22:45:35.17 +-1746056226,91746215995616656,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.348,1982-06-08,false,147458926.33,21:3:18.55 +608397784,24919001205410520,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.348,1995-03-09,false,481711187.853,21:17:22.25 +-575292771,37357256144624368,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.348,1978-11-13,false,543803094.609,4:55:22.58 +129430564,4285917316841840,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.348,1962-04-14,true,196576171.698,4:17:24.40 +-491752810,414086079439882,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.348,1972-10-14,true,156116783.691,20:5:41.24 +-964183801,24763141853882636,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.349,1975-08-24,false,76959091.0144,7:47:15.1 +1142560441,19850707381168804,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.349,2000-03-13,false,684489074.483,21:15:45.2 +1850363536,82582753408192992,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.349,1985-08-10,false,1075790807.19,3:33:5.2 +-371147493,55744616581453784,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.349,1964-07-17,false,913251604.968,8:35:23.53 +-582891613,81742404418952608,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.349,1977-02-03,true,905280205.924,22:52:33.16 +549224435,76010076734124480,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.349,1979-05-15,false,1306712086.69,19:10:40.34 +971387385,59458893380935336,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.349,2006-02-27,true,372630548.634,5:2:39.28 +1224114293,66854651439756280,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.350,1996-11-05,false,1422247427.58,8:43:8.41 +1355438726,33483495894517884,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.350,2005-05-17,false,1087041046.62,12:15:17.30 +-499284819,56119773199210808,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.350,2007-08-09,false,381351969.164,1:20:29.5 +578511268,20452011481311572,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.350,1966-02-11,true,393425363.139,15:46:55.40 +2084635866,62456170280568064,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.350,2006-07-01,true,52973427.823,15:32:35.14 +-1437457649,64859735634952960,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.350,1961-09-04,true,198969366.528,2:3:8.14 +1220108954,61219116995990952,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.350,1962-07-16,false,1203520226.87,1:16:26.28 +162008639,7839579018042737,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.351,1993-05-27,false,40374752.6924,14:53:58.20 +1887459710,18856017433427456,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.351,1950-08-10,false,338534955.845,5:29:36.50 +1082152698,83984674400884864,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.351,1998-06-11,true,32174605.5676,18:44:34.43 +64189458,57528605673650256,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.351,2012-10-27,true,257177767.531,7:3:35.39 +-438219140,45073760991374552,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.351,2003-09-12,true,426037945.736,18:5:47.43 +208316859,82522240322762496,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.351,1981-10-01,false,1243234868.55,18:33:26.11 +-1692710209,41776525566219344,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.351,1968-02-22,true,363378966.158,19:29:45.27 +-2040942100,71010036625356512,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.351,2002-10-18,true,1383906987.75,17:33:58.28 +1207858101,4634828848404470,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.352,1974-04-17,true,479232365.254,21:51:27.10 +826072406,56664153524995944,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.352,2001-10-09,true,44036653.1814,2:48:4.36 +-389155127,8476477011922268,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.352,1990-06-18,true,1082000582.4,2:39:6.13 +-1329626523,88199110285512752,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.352,1977-06-02,false,881969705.01,5:13:19.54 +737666825,67919295341463048,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.352,1962-11-11,false,703502113.594,10:9:2.3 +-927287814,61678804800630720,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.352,2008-03-06,false,842427683.99,11:28:14.53 +-2043543893,19996888042718220,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.352,2000-03-23,false,1185962780.46,11:27:23.6 +-1997968102,35288667010374708,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.353,1977-02-11,true,759035808.747,10:58:31.15 +417412510,43728809983714000,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.353,1997-02-14,true,713618674.264,5:38:9.18 +-1046515006,51591860413922872,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.353,1959-01-16,false,202845453.857,16:47:12.13 +1532354009,83645822521019488,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.353,1966-10-12,true,935865048.627,15:16:20.19 +1708369419,82229891423501904,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.353,1953-01-23,true,271675422.538,2:32:39.36 +1533899007,38105662598251016,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.353,1983-04-04,false,679437501.356,16:27:5.21 +-613644843,89954640137032528,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.353,1988-08-13,false,710788628.588,22:15:26.57 +243298489,24337445900956540,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.354,1962-01-08,false,1083616952.9,5:55:57.27 +-1206697048,35806987131557900,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.354,1984-03-24,true,1422678309.83,4:20:3.28 +1361110309,86512826392795840,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.354,1989-05-07,false,985273326.526,6:33:19.11 +-1572100107,89705752164544272,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.354,1985-08-09,false,1360347417.39,21:44:23.43 +2033579185,16221419318488310,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.354,1977-09-02,false,775854824.499,14:25:18.10 +-2070301352,58902059865932728,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.354,1991-08-17,false,754788319.884,13:4:37.48 +-327907275,5494426562720246,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.354,1992-06-03,false,43700715.8191,10:46:16.17 +1429237051,13042572793269852,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.355,2001-05-05,true,174604013.305,2:53:17.20 +-1841976954,54366836492911640,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.355,2001-09-19,false,76034952.3881,13:1:7.22 +988927569,17511128261822434,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.355,2004-05-12,false,436659713.053,7:31:25.50 +133570849,5696266093383803,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.355,1955-10-23,true,1195333696.1,17:17:39.20 +-1734062875,75869222168811392,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.355,1998-04-23,true,969184880.6,13:19:35.16 +2052980140,43846881745985856,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.355,1958-04-13,true,1086473587.13,12:22:13.43 +-512349430,82212969504791008,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.355,1957-01-12,true,224252313.322,21:40:30.11 +-1768308933,56979011337855256,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.355,2001-05-21,true,1267520391.18,22:30:56.30 +-1847320850,75124688786940784,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.356,2013-01-07,false,860677947.471,15:48:7.30 +298062858,35440499823901492,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.356,2000-03-23,true,1406440426.95,19:12:48.45 +395996872,54666773378435912,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.356,1959-01-07,true,854057913.877,19:42:32.5 +-2108884056,74715537116061392,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.356,2005-05-14,false,676469004.444,6:33:27.30 +611288553,61892601335167408,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.356,1982-04-27,false,1227494288.89,10:40:30.52 +443473198,81621411048840832,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.356,2007-07-06,true,716734248.874,17:3:36.52 +-814176898,69620052761721912,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.356,1984-05-05,true,182342709.75,22:11:30.9 +-527698382,85768609925637104,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.357,2002-08-25,false,1142213534.03,4:22:52.39 +-1861103999,52718515617136560,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.357,2003-06-08,false,1006150215.71,16:35:52.7 +-1809346762,13093127328527064,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.357,2004-10-23,false,749549575.287,6:16:16.43 +-515218497,82165792492043568,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.357,2004-02-12,true,297410833.097,13:23:7.38 +-962293807,53438832336505952,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.357,1959-01-14,true,1365731273.79,11:23:5.13 +1595266640,12867665961999156,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.357,1981-05-04,false,425280039.092,13:1:5.8 +-782264564,84708902463943888,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.357,1980-06-09,false,225621826.277,2:20:4.29 +2019038420,76241460867724080,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.357,2006-04-07,false,471496217.944,20:46:54.17 +-1234067225,16594396971411918,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.358,1985-05-21,false,102220365.234,8:10:45.28 +-1110550398,15849315108129516,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.358,1961-09-09,true,309627858.371,4:29:11.52 +964652029,22958419414620896,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.358,1960-01-02,false,896234637.348,20:33:46.53 +-21935258,26628302305665720,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.358,1954-03-01,true,663038256.718,13:36:39.37 +1691312990,75639002417601152,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.358,1953-03-18,false,1338728027.92,2:37:16.26 +-1433408731,73371668909820944,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.358,1966-11-27,false,292403741.081,12:39:8.14 +1689183180,33186335034304440,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.358,2009-11-22,false,177697053.351,1:24:42.35 +1608952697,61161530540080048,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.359,1954-10-17,false,1321114160.67,9:12:53.11 +1508711163,12987546898094766,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.359,2002-06-20,false,494862634.203,17:43:9.36 +1991025578,25708426691251068,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.359,1968-10-26,true,1128334667.21,3:20:22.53 +1830666419,80024076854706256,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.359,1950-06-06,true,74052251.1073,15:44:30.37 +-1619229249,29482355416570308,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.359,1964-09-12,true,520336433.062,17:51:33.12 +778462818,64886290547940176,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.359,1952-09-27,true,1218458225.9,12:48:51.8 +209415645,73256849433611520,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.360,2004-05-07,false,392631806.215,11:5:19.37 +1088715194,63948829011427152,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.360,1979-06-22,true,639943314.193,9:34:47.21 +-1335931280,59731436747619240,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.360,1972-06-08,false,508320767.507,13:6:27.47 +860549076,71750340315237592,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.360,2013-01-21,false,352373504.037,6:9:51.38 +1720315639,22156542495651780,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.361,1964-03-11,true,1070329450.15,18:29:15.57 +618785925,76389709418115328,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.361,2003-04-20,false,419252564.053,13:41:5.8 +-1291652172,90390748189264944,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.361,1964-10-02,false,1166650289.0,10:58:25.21 +62062923,36270178422501768,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.361,1964-06-11,false,461890330.222,17:12:19.6 +-203623507,11893208756284314,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.361,2011-10-02,true,1128558276.62,11:33:41.32 +-655227475,35573081473247552,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.361,1982-10-02,false,830689111.939,20:43:4.21 +-23968888,49640902479310304,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.361,1992-04-19,false,1075877869.57,18:5:37.21 +-1691477970,70209728367381616,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.361,2007-07-09,false,917664680.805,18:7:38.40 +-848424826,62081652356373560,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.362,2002-06-06,true,1049129341.29,4:49:23.12 +1162059393,13694215855978086,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.362,1963-05-02,false,305810677.473,16:30:27.23 +-213087738,21626936946669536,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.362,1989-10-26,true,618761910.619,16:5:3.15 +349834048,83098734345088560,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.362,1956-01-27,false,723502289.164,21:1:14.3 +1454675676,74997060056635360,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.362,1967-03-10,false,816314690.716,18:9:53.25 +708703697,56928404969429048,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.362,2010-11-03,true,765587044.287,21:14:53.9 +1011757834,1544796706629888,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.362,1953-02-10,true,700528933.98,22:57:52.42 +1145793769,91243183826545088,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.362,1991-05-12,false,781027141.138,5:47:32.51 +320307056,36290454690832408,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.363,1996-01-23,false,1257297378.46,6:37:45.30 +-1431968650,79122556926226272,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.363,1952-01-11,true,44882188.787,7:44:18.52 +-435725776,64633552699719376,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.363,1973-02-20,false,689959813.854,13:51:56.31 +976325484,52970528557373936,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.363,1973-11-24,false,381290921.966,15:10:45.6 +89062646,49320096734473032,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.363,1985-09-17,true,1176781563.9,14:56:55.16 +176444862,90762581082963808,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.363,2004-03-14,true,1226236431.37,20:33:1.42 +-382562264,46419605731663992,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.363,1967-02-05,false,1047920799.74,4:47:51.28 +-1826203213,23619719957156384,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.363,1970-06-11,true,1302514309.45,13:51:5.50 +1756909105,88800907461685744,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.364,2005-08-02,true,684605410.626,7:9:39.2 +-421048781,64179674268662512,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.364,2011-11-18,false,1330506805.79,8:37:29.44 +-711675553,1275508828769833,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.364,1951-02-26,false,697227127.313,1:42:9.45 +1604738523,51450532944424960,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.364,1952-07-10,false,9085806.29117,19:3:28.33 +-1208487113,62427913257225032,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.364,1979-10-09,true,1106419020.29,11:56:11.46 +2073131996,51189764688762656,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.364,1964-05-18,false,1289903699.01,17:41:52.14 +-917434267,1067822060820838,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.364,1999-04-05,true,438952753.782,4:11:21.27 +-549374525,76699758515812384,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.364,1965-02-17,false,838047981.963,1:27:54.24 +27037734,65581378975015312,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.364,1966-11-22,false,1036328946.18,7:25:58.20 +238320871,75482599130978624,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.365,1985-03-09,false,832513487.269,22:41:1.56 +492179648,82232175392440064,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.365,1986-05-04,true,844096853.839,22:6:1.23 +827715290,13092752516991478,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.365,1998-08-24,false,244075956.608,18:5:21.55 +1372345810,44521024209321464,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.365,1982-09-17,true,586979258.585,16:57:16.46 +-469424114,41379761050274224,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.365,1963-02-04,true,707469608.612,22:29:2.10 +-938495947,20119521268665416,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.365,1950-04-23,false,1430306867.46,21:57:32.17 +-1939245389,57404665864220000,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.365,2005-03-08,false,1266870989.4,12:28:49.32 +1545359812,48289542403499688,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.365,1963-03-01,false,828978630.542,5:1:53.13 +1019817594,72059209548444880,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.366,1976-05-14,true,1321073554.36,6:24:7.35 +1130846099,968526288195532,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.366,2004-07-25,true,365071815.261,20:20:22.45 +1353129640,22547395786650984,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.366,2007-09-06,true,1352552730.72,10:23:7.54 +-570252968,67824874902895056,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.366,1992-03-25,true,1081314328.66,3:48:26.34 +-1803864610,40087575843053624,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.366,1952-06-09,false,820427470.878,5:4:6.18 +-1381072190,32280283576698728,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.366,1986-04-22,false,504401823.774,17:55:11.57 +603702144,17391748230600142,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.366,1959-05-16,false,1003462516.03,20:27:23.15 +1268456935,80620691146827184,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.366,1972-11-23,false,1361507580.21,14:15:18.23 +-1775255711,13275485745498634,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.367,1950-02-04,true,1287317781.16,18:20:13.9 +1044963533,87261443570070496,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.367,1960-10-10,true,885218699.518,22:31:35.22 +-1759526016,79836799675950784,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.367,1987-10-10,true,1234824982.41,12:9:43.45 +-1997270115,81202860867404912,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.367,2003-02-15,true,496465145.64,14:22:46.54 +-1826278437,9640789710446388,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.367,2013-06-12,true,571905870.574,19:25:51.6 +-1397771824,64988172236330824,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.367,1968-04-01,true,454209401.067,1:22:43.32 +-985818543,52951279177037040,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.367,2009-03-07,true,308264532.046,3:38:45.46 +77365848,6222808629867397,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.367,1964-05-15,true,904877151.12,17:29:12.16 +152641723,47549477697113088,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.368,1983-03-15,false,129951877.375,16:16:46.27 +1567398505,89570271591341792,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.368,2000-05-10,true,163167316.1,5:28:3.30 +914855426,34448625598476636,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.368,2013-09-05,true,355185516.992,18:47:37.31 +363698878,41825304783849944,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.368,1992-08-27,true,1360231293.94,22:13:36.38 +425506464,39529541130545192,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.368,1957-06-26,false,927464638.109,20:42:24.58 +-52265657,9965484044701368,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.368,1956-10-12,false,414577535.246,3:23:51.40 +1148829696,40867683308263480,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.368,1953-05-09,true,766916035.485,1:15:38.55 +-2063747892,80285831514334224,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.368,2002-10-11,true,1331465994.65,20:50:48.55 +1929677259,48860509385993800,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.369,1966-02-02,false,587038545.452,9:33:2.13 +-416690939,59907375955492896,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.369,1980-08-11,false,1259593898.43,20:18:44.7 +-479552562,82138407584954176,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.369,1997-03-14,false,721651695.585,14:3:8.1 +401031725,48520332996149720,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.369,2003-09-14,false,352521816.211,14:49:58.58 +-1192063402,45843282150267000,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.369,1979-07-15,false,468531567.236,19:22:29.54 +1594873491,26575547242877768,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.369,2002-04-20,true,729697095.642,11:38:6.22 +-396979487,84490793636029920,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.369,1986-08-22,true,431084525.763,7:35:3.26 +987430619,26735766047309292,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.370,1981-08-22,true,1171253487.38,2:23:45.30 +672027528,17471360333764618,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.370,1990-11-16,false,441772453.012,5:38:25.58 +-1774642414,45010567538970616,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.370,2001-07-11,true,1118513534.59,8:48:56.40 +1450644832,53016772986887688,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.370,1994-07-17,false,283880620.864,15:19:57.3 +1997997901,19611546482523852,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.370,2009-09-09,true,767475664.485,9:57:7.14 +-861306216,90578675311567296,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.370,1978-11-01,false,1356943330.51,20:22:35.57 +-564188702,457842979752243,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.370,1992-06-26,true,1045008248.56,11:14:54.4 +-482526610,62076214665300080,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.371,1988-08-09,true,1032809449.21,3:44:15.54 +-317994361,28110442805984748,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.371,1975-06-02,false,618034430.855,6:4:20.1 +1978399725,15968755089962896,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.371,1952-08-12,true,201349218.611,16:6:53.42 +1371798487,56970080222018584,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.371,2001-03-11,false,939748960.766,12:38:32.17 +1139099020,21220983092454556,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.371,1962-10-05,true,16979933.5667,12:24:9.13 +649564362,56902298049916304,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.371,1994-08-25,true,259058419.771,22:51:26.6 +1441196043,13612820624952290,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.371,1979-11-05,false,1345428201.18,11:9:9.7 +590707012,62988317320328800,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.371,1988-09-19,false,595843809.679,13:43:56.1 +-21391273,51265787910364792,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.371,1969-05-05,true,49481602.3462,14:18:38.58 +-789937930,26558389769014856,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.372,1974-05-27,true,847782113.079,12:26:15.49 +-307733610,23186997310863608,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.372,1971-10-11,true,715257621.1,14:34:35.15 +481759253,90224539570431520,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.372,1955-06-13,true,1241818695.32,6:57:35.37 +-1827955535,38990612026901184,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.372,1967-03-17,false,85291982.5176,16:29:26.46 +-662528991,69099712548099200,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.372,1969-03-27,true,5151529.88676,21:8:32.45 +-1123757839,7844528446075597,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.372,2013-02-19,true,1135192653.73,22:54:49.50 +1541142190,49807521780892912,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.372,1984-09-25,true,854224511.375,4:51:3.57 +574433563,20641957292453848,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.373,1989-07-25,true,983323442.341,21:47:6.58 +1403994098,4334778495298888,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.373,1958-06-22,true,806749990.196,22:43:18.18 +820916123,24469458948059220,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.373,1985-06-26,true,989149168.6,5:21:25.25 +-727153588,22879916282683988,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.373,1999-02-23,false,1427455391.91,19:44:18.6 +-747970267,58008032357447080,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.373,1965-11-25,false,1377236780.91,18:33:4.36 +-202479980,71017099297145920,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.373,2007-10-15,true,1042097115.35,9:9:28.37 +686586054,40344709828080752,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.373,1953-11-11,false,945031147.065,13:36:38.37 +1968922414,30453870367122400,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.373,2001-01-18,false,252243163.013,13:3:12.16 +1550270673,86011806067593920,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.374,1997-05-03,true,842622558.1,14:16:17.30 +1336007800,84267955628252240,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.374,1981-07-20,false,664595491.369,11:46:53.21 +-2048643105,21560110742925568,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.374,1964-09-04,true,189421486.715,8:26:42.23 +-2142771076,616256601560320,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.374,2011-08-03,true,885378889.248,3:53:49.40 +643376678,34875940995087728,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.374,1965-03-12,true,86072021.1222,21:8:7.54 +503159378,61865818032061760,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.374,2012-10-25,true,110978615.251,2:48:45.6 +154045765,23398269731765044,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.374,1959-06-04,true,1410874937.78,20:29:18.30 +1009896485,6066714731017390,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.375,1955-03-16,false,58234004.912,17:10:4.18 +-1758142100,14811069675169312,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.375,1993-01-05,true,398174725.43,11:35:57.5 +1845544644,11295711695422146,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.375,1979-02-16,false,674907984.658,14:1:13.27 +-1049540051,28393682436953448,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.375,1969-01-15,true,1286556005.99,16:21:48.5 +-1643918980,12974399773828650,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.375,2006-02-19,true,256266521.766,8:47:17.16 +1604368540,47580293861962760,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.375,1962-02-06,true,1065331300.25,1:6:22.26 +712289413,79165763528856784,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.375,1984-07-13,true,702774049.295,16:12:20.33 +-763848238,23774971107493020,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.375,1990-06-09,true,521057535.945,19:53:19.53 +1987899487,34494074042188884,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.376,1971-07-16,true,212927244.748,12:17:35.27 +-1629176849,63268185789033184,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.376,1985-04-09,true,615583308.157,22:1:43.8 +-1407371126,36943105540733112,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.376,1962-04-12,false,388488733.32,1:44:46.57 +-1048204870,80394134174283424,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.376,2003-02-13,true,20787372.2775,12:42:2.28 +-588692334,91837370775373744,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.376,1951-05-01,true,794332662.09,7:49:38.33 +-381996905,41574734493563600,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.377,1993-02-03,false,453165690.508,11:3:20.3 +534432012,26638164374683608,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.377,1989-04-23,false,928522133.015,14:52:49.52 +-2088830083,36832164395186648,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.377,1999-06-05,false,262315954.27,9:5:39.2 +2125449426,85213735730024352,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.377,2001-07-04,true,240416511.536,3:40:38.13 +-405135869,8686544103564677,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.377,1971-07-19,false,1024566183.24,9:33:30.7 +-878774129,72742579955351200,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.377,1979-07-19,true,735366475.54,2:5:49.38 +-817589874,17098478206419374,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.377,1995-06-05,true,594818095.599,5:34:16.54 +831248490,77633628833575872,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.378,2013-04-27,false,851769081.623,8:4:25.54 +-310206516,26935082892426396,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.378,1977-02-04,true,420338019.846,1:31:25.49 +-1323352306,45761314602862456,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.378,1972-08-22,false,852576425.982,10:7:31.10 +355279288,34102286041539092,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.378,1990-07-11,false,1171409828.82,2:25:17.44 +2007935126,44301070867064816,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.378,2000-10-02,false,748944249.733,12:32:30.25 +-800021846,61654564798722328,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.378,1957-10-11,true,1267844519.31,17:4:50.57 +-71688081,1150951440485939,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.378,1976-04-18,false,726794450.966,6:15:19.54 +194880504,47702501508088952,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.378,1966-08-25,true,1199420528.42,17:28:22.41 +-2144203426,84953503115681424,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.379,1988-05-16,false,86986904.7603,21:3:20.44 +-1347192683,85743746419789824,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.379,1966-06-14,true,95258018.2273,1:9:7.25 +-205692330,41251982375217872,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.379,2009-03-01,true,847132214.069,14:6:9.3 +974412955,4138740553994332,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.379,1984-10-17,false,315626701.702,12:54:45.35 +-1239201879,4787293688657285,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.379,2011-09-17,true,1009918893.39,21:30:23.14 +1091278579,38803511956171696,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.379,1978-01-13,false,847034494.949,21:47:49.27 +306818718,48155625772720440,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.379,1997-09-18,false,1332638333.72,4:17:18.7 +1868101717,8293497074949489,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.379,1974-06-04,false,1083657211.44,15:3:43.46 +-669628775,89370666536098976,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.380,1992-03-01,true,643392756.136,13:14:39.3 +-883846419,13504721451322368,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.380,2005-11-08,false,1271007014.86,9:2:23.30 +1944821544,20582533021732372,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.380,2009-07-19,false,1293058576.49,16:1:43.36 +-45277166,40425719169077256,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.380,1983-11-05,true,83210941.1463,4:53:36.22 +1344166125,39256098026120672,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.380,1956-06-15,true,1286571136.22,21:47:15.29 +-2083732876,27837012700368968,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.380,1977-07-01,true,511728703.993,11:33:38.16 +-1071838333,64336881061170360,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.380,2010-10-20,false,1263949392.44,17:11:4.3 +1658821749,66456485654441840,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.381,1950-08-07,false,764925419.817,7:45:27.11 +1384669169,23243808091013540,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.381,1950-08-11,false,1144786631.26,5:14:17.43 +933490323,91851110532284688,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.381,1972-11-20,false,1209593617.17,8:14:55.32 +136232093,81697509936259600,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.381,1966-02-01,true,239829957.02,14:26:28.8 +-2074685805,86446320373341744,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.381,1951-09-07,false,965989054.039,12:14:25.12 +1113607362,37916058895790488,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.381,1980-05-22,false,762299282.826,10:40:38.17 +-1202414969,29381995470129460,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.381,2006-11-19,false,55940557.0842,9:52:40.31 +-1165405335,77128621499926480,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.381,2004-11-05,true,296735630.008,5:2:26.40 +695495954,53450116636670400,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.382,1983-09-16,false,123016255.859,12:15:36.53 +2002321150,45104452491231680,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.382,1985-10-14,true,237857819.654,12:30:17.22 +-69773821,90487122468037280,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.382,1982-07-27,true,803477287.787,7:1:4.12 +492922159,926123797073305,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.382,1968-07-07,true,1171524510.78,10:23:47.45 +1364128958,61530842179130376,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.382,1957-11-05,true,1286068877.5,4:2:43.34 +-1466049325,26525239625329264,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.382,1971-01-23,true,583735134.034,6:36:1.52 +1542036171,59192743168087144,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.383,2005-10-27,false,273886721.284,3:47:43.19 +1966731123,31800601087252704,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.383,1958-09-13,false,48123976.2543,18:10:35.4 +-217899504,4280646703831152,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.383,1958-02-02,false,57112256.7519,15:55:49.29 +-736910582,80749665091798096,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.383,1986-04-06,false,1191604590.04,5:33:40.20 +-1019456374,2800421403382282,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.383,1965-11-08,false,1075154134.22,22:39:23.37 +-841871058,12269836918661100,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.383,1991-07-16,true,99954041.7715,2:36:55.14 +-1177095073,64958648681006720,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.383,2004-04-18,true,499217768.947,7:22:7.12 +-1571774990,20081963220737712,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.384,1989-05-08,true,616228698.665,7:21:35.53 +1906808734,68239459452603248,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.384,1969-01-27,false,761886148.281,3:31:53.46 +454601531,43717886022332792,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.384,2005-03-17,false,361465137.238,2:50:55.6 +-1088878842,45032702173332488,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.384,2013-11-04,false,1266203384.68,11:51:9.5 +-2103900400,38860294908919736,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.384,2010-11-20,false,1347702891.71,7:21:28.51 +-655459768,40360602147502160,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.384,2010-03-24,false,543472512.39,2:38:45.44 +2115885185,46769129239323784,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.384,1968-10-14,true,242970242.107,17:7:2.42 +-1255435754,78157482733207776,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.384,1992-02-11,false,946663726.334,22:42:32.40 +-1091737404,3834695774370898,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.385,1957-05-15,true,1268506202.67,5:58:56.32 +737375615,20162187083325932,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.385,1988-05-26,false,1106528880.11,11:6:20.41 +-1651108886,16625174475467684,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.385,1996-10-27,false,316788549.526,22:10:27.12 +-295420003,10335644792997222,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.385,1955-10-13,false,1430823258.71,15:17:7.51 +-787644835,42543997991225920,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.385,1992-03-03,false,754747894.713,7:33:48.1 +1118215039,80725153036979280,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.385,2006-10-03,false,1323006518.85,3:35:32.56 +-1818715759,80704378480519696,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.385,1996-01-08,true,172738990.406,4:51:25.17 +-439469011,68939049384171584,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.386,1955-07-20,true,1319914428.46,19:45:58.38 +-256624270,49971380046528504,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.386,1953-09-17,false,945830686.297,6:49:43.44 +1792298409,54892235174242120,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.386,1987-03-02,false,420466787.556,2:43:7.30 +1627481796,38224242091923752,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.386,1999-09-25,false,849221127.943,5:37:40.30 +-686547260,24439129108315588,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.386,1985-08-03,false,270622014.493,6:3:25.57 +2000576896,90325692383208432,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.386,1983-05-13,false,16208166.1412,1:36:27.10 +1216910988,16262845443391826,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.387,1969-03-23,false,8544279.22394,2:50:26.13 +1645169236,76001914035734048,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.387,2004-01-19,false,241017243.428,17:45:57.21 +-1043193946,48405441404448168,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.387,1993-10-15,false,762730753.749,11:51:35.36 +-1045582365,33156560676438212,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.387,1968-07-23,false,1226067245.42,21:35:51.43 +-1368934840,34945252073020428,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.387,2002-02-22,true,545345500.776,12:5:40.24 +-1322301336,57962558701225768,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.388,1997-08-15,true,796531321.335,11:4:39.3 +325618860,46898979245564624,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.388,2002-01-22,false,126689296.858,17:30:17.5 +-1389349643,30989125815542672,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.388,2003-01-16,false,473535116.266,11:26:39.50 +1859166344,38044796325388016,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.388,1957-06-02,true,928279629.684,21:1:24.42 +777368180,73585815970879312,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.388,1952-07-17,false,601761218.735,21:17:25.15 +2085293257,10331737660523550,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.388,2005-06-14,true,48329283.4799,15:14:8.44 +-1778805812,19394119840300032,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.388,1950-11-18,true,1103458881.34,2:16:38.49 +-1725251504,13785545935388222,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.389,1971-08-04,true,969974136.689,3:34:52.6 +-1281552132,88404693574152144,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.389,1950-10-11,false,134456139.38,14:38:29.44 +451330411,92105363372149840,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.389,1962-04-09,true,1113743662.68,5:54:19.26 +-341485142,75467558399017424,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.389,1965-09-14,true,967014336.394,2:58:34.25 +-1556872995,89059330591722464,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.389,1972-07-27,false,1348030613.42,13:34:30.55 +1283883362,41025689994525208,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.389,2003-10-27,false,827801228.542,8:38:15.42 +1784255976,19600067982486784,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.389,1954-05-05,false,791061846.865,1:21:47.48 +1049262574,88424753449523488,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.390,1987-11-23,true,1337661137.08,17:17:54.41 +1569649250,32384278033775564,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.390,1975-11-06,true,547084617.569,10:15:47.14 +1732572278,77805226137128240,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.390,1956-11-12,true,218550147.029,17:41:40.31 +1345608996,63306669058853696,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.390,1956-11-20,true,1333754958.59,1:58:38.14 +1364618302,6591647891209011,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.390,2007-09-01,false,950609863.72,2:32:16.43 +-1582222500,69491022128239280,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.390,2010-05-10,true,950266828.987,1:56:50.24 +1132800489,87490833859740560,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.390,2012-06-18,true,814207350.443,3:49:54.11 +124281232,26554322690417900,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.391,1997-07-12,true,721438223.104,2:58:46.9 +1532089297,51196162099811640,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.391,1957-10-19,false,1295547254.01,22:54:16.44 +154229240,91642582757430544,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.391,1971-09-03,true,1215580092.74,4:38:4.49 +-381290462,2831718869273825,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.391,1975-01-16,false,34874241.2318,20:52:13.45 +-2022660915,32987416289337232,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.391,1956-08-24,false,1264681965.81,12:12:2.18 +2075526412,13405916522295644,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.391,1990-03-02,true,1368484886.68,10:20:36.21 +74883352,77020472369711472,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.392,1956-10-23,true,314363826.504,22:2:57.38 +831452010,4956724501094441,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.392,1951-11-24,true,115355829.013,6:46:10.20 +-812219330,48894858079582264,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.392,1997-02-04,false,1313797875.96,7:9:19.48 +224657250,69974749522136656,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.392,1971-08-24,false,670900822.628,9:11:43.40 +-59894467,78272353568188288,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.392,1957-07-04,true,1421973532.51,2:14:14.26 +-260705086,41051305057840832,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.392,1968-05-02,false,1235446049.79,9:46:45.17 +-1530369886,67458204465818832,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.393,1976-10-09,true,378754691.335,8:46:26.45 +159090540,37562495760250512,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.393,2003-02-26,false,80866469.6611,4:30:45.32 +-1706884229,68568895452030592,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.393,1980-02-08,false,208759766.517,20:18:36.10 +1729075383,7346293296017930,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.394,1984-11-03,false,584725712.693,4:42:20.14 +-24214560,43027348973245632,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.394,2005-08-09,false,237586688.726,2:39:43.30 +570547030,66991376954571416,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.394,1987-04-15,true,841058051.302,4:19:26.57 +-1546941478,47261369175275368,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.394,2011-05-27,true,746449782.502,19:54:15.50 +-671724558,1014925997162403,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.394,1989-06-04,false,1019260618.4,9:6:26.30 +1163660546,44973143236677592,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.395,2003-08-10,false,1396462582.4,13:15:9.6 +189117886,22891407284001824,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.395,1952-10-21,true,1427784866.97,14:3:13.8 +303462851,67563364207057296,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.395,1963-06-05,false,333785917.905,22:42:56.21 +-652409762,17928618821692980,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.395,2006-06-13,false,951673191.355,1:24:23.17 +289764309,62321333408546712,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.395,2005-04-14,true,259970429.312,17:37:48.20 +1621719973,51183014427261888,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.396,1956-10-09,true,398937248.313,7:45:44.26 +-1044878037,77158759318762352,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.396,2003-09-09,true,60845853.1998,8:9:50.20 +-1830340029,8662044688127969,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.396,1959-11-14,false,674379632.628,3:44:4.51 +69722543,89666997414133632,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.396,1967-01-10,true,857001658.464,7:10:14.29 +-1400955413,81122091731780592,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.396,1977-07-12,true,1208078323.61,17:14:54.20 +-1536093749,9088840199562598,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.396,1958-05-08,true,1286929407.06,15:46:31.58 +-1355772040,57937857992766616,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.396,1958-03-05,true,349188158.405,8:28:28.34 +-1169489773,66921326584184680,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.396,1957-02-06,false,254911853.694,16:18:28.22 +-831664664,43268421363384568,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.397,1977-08-04,false,1114332815.95,20:35:57.7 +-1776914611,18650674329400524,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.397,1954-07-15,true,69819876.275,8:7:11.6 +1919919251,64129376441172216,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.397,2002-04-06,false,1020084602.89,20:13:10.10 +81649631,31953700456680580,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.397,1993-02-24,false,437260002.631,2:31:58.42 +-1223738853,9305374903240612,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.397,2010-08-26,false,865905369.705,10:12:29.57 +-1032733790,43293264238688000,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.397,1977-07-22,true,1157896605.86,21:7:14.11 +10762987,24455375078291292,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.397,2008-11-24,false,1216449287.59,18:41:21.50 +-1957633166,49912435091947944,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.398,2012-05-10,false,899187622.499,13:58:42.24 +-720474661,74056891784722832,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.398,1976-02-01,false,1128184212.07,8:22:8.23 +-1423580634,14142839041389958,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.398,2006-10-24,true,141515782.055,6:49:53.20 +1344211260,56131062375912304,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.398,1960-02-20,false,1118386107.13,19:55:7.19 +-1522823409,24177567072552408,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.398,1975-02-02,true,58395163.9651,16:23:13.28 +2041666846,45204918771949008,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.398,1958-07-18,false,911227661.992,14:32:29.51 +715704617,82524415696485280,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.398,2009-06-02,true,1070739212.05,9:36:42.51 +938939127,52520853461387152,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.399,1971-01-24,true,330150203.753,5:1:54.37 +1900992833,42611609232649176,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.399,1958-08-23,false,63347874.6105,10:10:39.29 +-1643285190,7029712144566723,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.399,1970-01-25,false,900249668.532,17:57:43.46 +-640726554,74053310636948464,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.399,1980-06-08,false,681810716.289,21:13:29.15 +-1222087652,4698805569927557,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.399,1969-07-25,false,1372313906.41,18:11:58.43 +917781209,41506965506233848,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.399,2010-11-07,false,1046892798.67,5:1:37.13 +2063970534,45774254153662520,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.399,1977-05-26,true,1245460261.11,20:24:50.9 +-1221414249,77497398390839568,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.399,1969-11-03,false,1339519175.5,17:49:53.42 +2028941034,1725262344065710,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.400,2011-07-27,true,786136208.444,16:55:19.46 +1449430517,42778276576346728,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.400,1983-09-18,true,139155358.182,9:39:38.44 +-561502763,39758433531931144,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.400,1962-06-22,true,917994962.784,21:11:57.16 +-1861350462,21856819214996500,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.400,1954-04-27,true,1169611984.53,2:31:8.34 +1761167951,15208695919587380,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.400,1954-01-17,true,521223436.963,2:47:51.31 +1377700815,83779288712841200,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.400,1993-03-20,false,666311580.849,9:41:49.44 +1709934457,74423536303868720,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.400,1952-04-01,false,1013371688.92,12:13:56.46 +-1966817071,16653187926034800,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.401,1961-08-24,true,1222054557.96,13:24:34.30 +-469371431,71079651076631024,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.401,2009-02-06,false,625634618.579,3:51:5.48 +348856435,76196788431137392,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.401,1999-11-11,true,355264178.833,16:23:31.52 +-605994968,25477496868349032,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.401,2012-07-09,true,746145308.243,1:30:17.57 +-814482515,69569840508645112,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.401,1954-02-13,true,947677132.165,10:26:56.48 +-35142959,3009877931096985,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.401,1982-06-15,false,463853729.619,13:11:22.13 +965281763,16255052934879222,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.401,2000-04-16,true,1303815647.95,14:40:21.1 +-1518834710,90709071518206848,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.402,1958-03-07,false,583696049.444,6:34:6.34 +-41362092,16058899058397236,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.402,1998-07-26,true,656774073.136,20:2:33.23 +2102522026,73985080721046176,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.402,1957-06-27,false,643292363.623,3:12:11.17 +-534062665,43139392000926576,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.402,1964-07-08,true,515240011.83,6:46:13.20 +-678288892,45432051685500240,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.402,2006-02-18,false,1226739799.98,15:14:37.37 +334839614,35684378855557756,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.402,2006-09-09,false,888752500.678,12:37:15.42 +2046378249,2594655207865579,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.402,1991-10-16,true,602769109.091,2:23:50.8 +514332060,74075825598099568,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.403,1959-04-20,true,746087869.797,7:13:52.48 +1423218872,53905955261722048,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.403,1966-06-24,true,302636781.295,20:46:49.19 +-505464034,37614902878833088,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.403,1987-11-03,true,822216865.028,2:23:52.27 +-1359510142,46303467333929784,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.403,2004-07-01,true,860928336.794,20:30:26.2 +1912626836,36161620611797544,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.403,1975-07-24,true,201901237.548,14:20:21.2 +-2022892880,76834936417161280,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.403,1970-09-19,true,1017235124.76,4:51:51.49 +1414323236,42149545167425160,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.404,1969-08-11,true,527992887.493,21:58:18.13 +-1961948096,33418420142204180,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.404,1966-08-22,false,1392789067.11,14:4:1.55 +-827009997,12120486112024914,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.404,1953-01-25,false,173810866.449,8:56:29.27 +2029854206,62732434505153864,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.404,1971-08-01,false,1346897153.29,6:28:55.19 +2079281386,78350221059971056,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.404,2012-10-27,false,40358431.0361,15:9:38.28 +1326835394,72372208003634448,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.404,1978-06-26,true,825816334.685,16:51:10.34 +-1388688860,71618347607907064,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.404,2012-11-13,false,970841266.042,2:45:19.51 +1642774701,69702613910388176,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.405,1965-01-27,false,1149072747.62,18:8:14.29 +-1881642049,76725230494794624,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.405,1972-04-13,false,1139269797.69,5:2:25.52 +309624908,83367058786272880,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.405,1967-08-17,false,947198978.79,15:9:35.20 +-641907630,57205051847114432,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.405,1979-06-11,false,846862033.141,7:13:46.47 +1036846223,65034814302288256,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.405,1969-09-07,false,1090292318.46,2:15:40.22 +1026350202,55216404742080904,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.405,1997-02-18,true,584461075.406,1:30:24.30 +-424603594,2611381608397506,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.406,1955-10-15,true,1398672609.15,16:32:45.43 +964316076,83865486648266928,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.406,1975-05-04,false,1271507038.97,6:25:38.48 +1787513922,20009453560752580,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.406,1991-06-03,true,194305491.8,14:22:30.5 +2108092357,50165943621479528,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.406,2004-05-19,true,1123148619.42,15:16:51.2 +-668813759,72549892048414480,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.406,1972-05-22,false,1325924395.63,8:9:47.41 +1730661543,57748469893089816,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.406,1965-02-07,true,1032017227.59,8:28:1.9 +1670724849,49587271311320440,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.406,2009-03-20,true,1362189726.22,7:25:39.35 +-25207261,73904499060459984,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.406,1954-08-09,false,1018599868.93,20:43:13.41 +1246969520,2220568733561211,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.407,2002-03-11,true,262190168.814,8:29:16.19 +-1388838824,62933875653600336,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.407,1996-07-21,false,477631090.064,2:24:44.3 +-439008734,43464088584924088,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.407,1957-03-10,false,432747378.199,5:19:32.53 +1711773483,77117966888625904,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.407,1951-09-03,false,429750512.53,21:39:22.58 +-1686983164,65836561710269720,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.407,1953-01-04,true,543174113.494,3:50:56.14 +1547939294,18891042573316228,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.407,1996-11-20,false,716233424.625,12:46:51.5 +-978757903,86373775989951600,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.408,1951-06-18,true,37693507.5831,8:30:52.4 +357506516,54806601390856112,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.408,1982-01-13,false,1274226276.86,11:55:14.51 +-69172538,81101720822693136,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.408,1969-05-05,true,1041771556.0,16:56:19.42 +189795663,45733499200544032,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.408,1991-01-04,true,708215539.666,6:42:39.56 +-632244417,50688396843446584,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.408,1982-11-18,true,797276605.318,16:39:36.33 +-925180049,14794020287606590,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.408,1956-03-24,true,1192121501.98,3:42:29.41 +860872570,30399408696437484,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.408,1976-02-23,true,888289733.793,1:23:51.47 +631875830,80575630490405424,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.409,1955-06-27,true,391467930.959,21:23:20.16 +-688533432,27059528882187808,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.409,2004-07-13,true,717091202.859,16:25:55.16 +-41755226,79684446915104048,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.409,1977-11-09,false,541094732.438,3:21:4.29 +-183586341,74025325957095888,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.409,1968-02-13,true,384790304.221,2:42:12.16 +893522863,65004468644430000,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.409,2007-03-27,true,842265801.555,7:25:15.44 +-1482889400,20837629386473648,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.409,1992-05-12,true,887493115.34,17:10:33.21 +-170837650,76646067738541840,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.409,2011-07-24,false,520368411.506,19:5:14.17 +1186859034,70058908682090952,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.410,1968-10-13,false,1034792933.41,11:55:24.5 +1205767996,10264163511715830,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.410,1965-10-05,true,1242630935.23,6:36:54.38 +865172650,21519764794074780,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.410,1955-05-05,true,1385189994.0,4:32:38.44 +2079980401,44590551637191616,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.410,1966-04-02,true,48723697.0842,7:23:8.43 +-963992389,77205280693079328,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.410,1981-05-22,true,451881910.931,11:31:1.18 +-1246355879,66056244544637136,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.410,1983-04-19,false,933858417.315,3:18:3.3 +-1418340804,87168275553913680,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.411,1997-03-15,true,1043450575.98,7:8:22.23 +-1006781774,66569701313788936,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.411,1977-11-10,true,461893506.339,1:10:26.32 +1199108759,86913495210387840,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.411,2011-11-10,false,1096812058.05,20:42:5.11 +1863789146,82771538853414624,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.411,1951-10-14,false,1154237420.95,19:12:39.55 +1799695276,79310223099345776,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.411,1991-08-10,true,984248078.382,2:51:8.50 +1277364573,4664667177455647,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.411,1982-02-09,true,987949248.031,17:20:18.9 +349392420,76723117105538320,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.411,2004-08-12,true,189304482.611,4:30:36.34 +-1274757896,8470225968887624,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.412,1957-03-09,false,1091618715.43,14:49:14.8 +-1149747782,53094948765441960,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.412,1958-07-22,false,1350975931.57,18:4:37.36 +412615404,61213595650063880,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.412,1970-01-16,false,409116777.344,17:41:34.51 +955348170,57263188514104896,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.412,2002-07-21,false,1005933751.22,19:47:50.1 +1429714118,26949446176314544,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.412,1955-06-05,true,1039356944.32,7:9:31.34 +619634723,39629769106385400,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.412,1957-08-03,true,493648686.231,5:38:11.13 +922732782,57645606442689312,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.412,1961-04-04,true,1199181425.5,13:43:29.4 +1293973755,24729535382546656,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.413,1993-10-13,false,207890603.342,22:57:17.12 +832422924,21857976338834792,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.413,2005-10-05,true,261699258.183,11:52:57.27 +-1106491005,13696160383163300,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.414,1959-04-06,false,144576126.776,3:22:20.15 +236249628,72711991446214304,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.414,1962-01-07,false,1308864206.17,17:34:28.4 +829072368,33031436308114608,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.414,1965-11-26,true,442054696.075,12:15:57.45 +-487078186,79579373411759440,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.414,1998-01-27,true,490585336.119,1:26:12.56 +-48931941,85325084284685648,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.415,1995-06-01,true,954791482.816,19:25:24.18 +-439391753,2176143374543954,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.415,1957-11-20,true,1338582673.41,20:37:57.57 +-796101878,16320048302693172,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.415,2006-09-07,false,83067646.4916,4:24:21.55 +318037227,30744477671269900,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.415,1983-06-10,true,75712160.1154,8:35:54.7 +-1617581687,79870727571763504,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.415,2009-11-04,true,537130324.977,10:11:40.47 +-117386556,26894506440660736,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.416,2012-10-01,false,707087172.965,3:42:15.33 +789537817,55197652198105560,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.416,1967-09-20,false,996326589.199,15:36:14.26 +-1858701423,71054488675781552,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.416,1990-03-01,true,780345551.917,11:22:1.1 +-2139508250,76520062564858928,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.418,1962-08-13,true,1248695809.18,13:29:43.47 +216893573,17450749493211586,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.418,1999-08-27,true,357606365.452,13:54:42.51 +1043129089,63429541567945680,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.418,2002-07-17,true,723767773.696,22:52:14.10 +101686182,91670385410906864,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.419,1993-04-20,false,168246197.284,8:25:33.50 +-52682971,81271356490709152,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.419,1972-04-01,false,818658837.023,17:29:27.14 +1149997953,26968511951055936,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.419,1985-01-07,true,527267518.471,2:36:58.28 +-558196117,1312565660269998,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.419,2001-02-25,true,633995900.784,15:29:11.11 +657823843,45131542311455824,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.420,2010-03-06,true,31228492.4932,17:32:37.41 +-248593704,64032143352006976,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.420,1983-03-20,true,360111176.073,17:10:47.54 +-2014736783,17709574074074922,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.420,1951-10-07,false,929090524.187,8:1:37.24 +560501494,85801372251805872,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.421,1955-02-21,true,900732662.241,17:1:31.55 +2104154007,43631876504605584,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.421,2012-07-07,false,1363543214.64,18:40:52.10 +870783309,50653541522471176,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.421,1986-11-07,true,1325125770.1,13:2:2.38 +1985509057,58988866933628800,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.422,1998-08-08,false,1371274760.87,19:26:14.22 +-560164043,18169226791721812,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.422,1965-01-23,false,604303070.886,6:33:47.55 +1391910998,16650113294354166,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.422,1967-09-14,true,916539300.16,19:27:3.28 +1482942038,82744539693382736,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.422,2010-09-25,true,1409162974.17,11:6:8.51 +1217328080,65438308705639432,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.423,1978-09-12,true,51517614.8948,20:34:31.43 +2095287728,40581650130413992,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.423,1988-04-25,true,1272501179.81,4:39:38.39 +1582658418,38623363770773616,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.423,1964-11-25,false,1034589807.13,6:34:19.45 +1001930036,60033152459346504,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.423,1996-02-19,true,356311581.63,13:16:41.42 +1999284682,55985949283092768,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.423,1982-10-27,false,113403350.38,21:12:8.8 +-1600024196,17169876529196544,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.424,1959-04-22,false,101866457.519,16:17:47.29 +-842095484,28941539898959380,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.424,2008-02-01,true,636174830.101,15:41:50.31 +-19389558,28672524936088148,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.425,1950-11-15,true,1124664169.62,3:32:2.58 +-851080996,85605553617770768,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.425,2000-07-17,true,820835537.643,13:26:50.4 +-1242389917,30906008607547852,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.425,1959-08-19,false,56910480.3415,3:39:41.38 +145105673,57945273071425152,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.426,1985-10-09,true,944309920.748,17:22:13.10 +1599800314,7654286841522248,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.426,1999-08-06,false,410581477.873,4:41:25.8 +-1853150718,20250047419213240,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.426,1998-08-07,false,979584938.866,4:47:22.1 +682611076,85218275004525520,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.426,1992-01-12,false,885194224.345,9:56:28.22 +565445591,26110227550429944,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.427,1992-02-24,false,383374403.786,8:43:3.15 +-1303134925,76765948415237232,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.427,1990-04-06,true,151238084.871,3:52:37.31 +-30105005,57360701309155136,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.428,1999-11-12,false,226339746.868,18:57:23.1 +1804733989,17802714199423612,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.428,1982-08-10,true,553536257.355,14:24:43.14 +-1137781231,57684872438794792,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.428,1985-03-15,false,904144699.987,3:44:31.29 +-621954282,43440527805594776,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.428,1994-03-12,false,43848127.4826,11:51:32.8 +-722114638,55516817824782800,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.428,1970-07-19,false,1212616377.15,1:29:45.14 +592124521,35918730928845508,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.429,1974-06-07,true,588944192.27,2:13:11.27 +1787465978,8639948926542070,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.429,2002-03-05,true,498215071.679,10:31:6.29 +765887953,63265737045532424,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.429,2006-08-25,false,369528982.252,7:17:32.32 +891922215,74298761201584320,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.429,1957-11-24,false,1218934951.29,4:15:32.23 +-703293840,49880898545575904,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.429,1971-10-25,true,1254864147.28,13:38:39.40 +-1557689533,69686616674724672,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.430,1984-02-13,false,416694961.648,14:9:39.27 +1112684783,49346754974137072,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.430,1960-08-12,false,1115126382.01,5:53:44.41 +269627264,61691196910581184,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.430,1960-08-22,true,23512922.0912,7:20:25.1 +-1032624489,50493508034576040,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.430,2009-02-04,false,1253221831.79,4:8:21.31 +1386265936,41090547989528096,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.430,2000-09-20,true,753884514.258,6:14:6.24 +2036268950,39434886525282240,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.430,1951-11-05,true,1398198735.66,1:48:3.47 +-1264054239,31444757906315676,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.431,2004-01-11,true,1073495224.13,12:28:29.31 +-942367046,65966436757408184,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.431,2008-07-01,false,1196357203.52,9:23:27.29 +1583993028,72631796417237296,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.431,1953-02-18,false,91126910.8513,13:31:20.10 +-259366121,51781369743514208,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.431,2006-03-16,true,1322609532.19,5:49:24.22 +1499688897,9793737580414158,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.431,1967-04-06,true,237938728.376,8:28:26.24 +-423461448,55925392425444608,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.431,2006-11-01,false,306798116.705,2:52:3.26 +1192066795,88346825406437600,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.431,1951-06-22,true,733102722.999,9:3:43.51 +1728576094,56028892888976168,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.432,2000-08-26,true,426999711.772,6:12:2.28 +-913039642,24200474995747196,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.432,1963-10-21,false,777030170.698,15:20:23.7 +971048623,15368821786163436,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.432,1977-07-04,false,62167605.0966,12:57:39.41 +-1957570929,77390322601712720,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.432,1980-11-24,true,941113830.819,8:43:19.27 +1399412244,76729096164433552,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.432,1974-11-13,true,1236113539.66,17:44:9.33 +312098752,71159990387333776,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.432,1982-03-20,false,593833917.886,17:8:28.24 +-931765316,74829434427960544,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.432,1989-11-17,false,761229638.947,5:28:18.2 +1057325817,88244654202879424,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.432,1972-02-22,true,826649170.696,11:29:15.27 +2115809977,45081263625196696,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.433,2012-05-04,true,711851604.579,4:4:27.10 +1674307054,78591017782890544,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.433,1973-10-12,true,976654043.552,18:21:32.20 +2008289426,14577850769652000,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.433,1955-07-19,true,1324570564.92,1:49:21.27 +1159454551,91520494708389344,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.433,1960-06-21,false,490569839.629,22:57:40.52 +1700759551,59216053746057152,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.433,2010-05-07,false,1273414131.39,8:10:44.48 +-897990599,44436256857684088,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.433,1992-09-07,false,1128233606.24,8:5:48.49 +1969389729,90759143901705216,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.433,1970-02-07,true,163498692.05,1:58:24.39 +1578361110,33230374888537764,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.433,1987-05-07,true,319257678.353,22:32:37.29 +1713304332,82601739902170688,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.434,1985-04-07,true,1274886559.41,5:22:33.36 +-1455773449,6585335444399647,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.434,1986-02-05,true,999839336.182,2:33:39.50 +-30559790,55494480616590136,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.434,2013-08-11,true,814111976.58,10:30:57.51 +-796177249,58735141468423776,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.434,1985-04-18,false,497979378.931,21:58:10.14 +-751300382,31370599633962976,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.434,1985-04-02,true,1135097295.49,2:7:32.28 +622925640,42050575899873824,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.434,2009-04-04,false,979856846.351,10:19:13.52 +1251862812,8865981900233677,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.434,1966-09-10,false,725473500.11,7:50:29.21 +-2012963910,16961466615476040,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.435,2001-04-20,true,1191393877.09,7:52:29.35 +2106196226,9961164293493894,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.435,1954-05-25,true,329460341.02,21:42:15.43 +893420154,31523945780441128,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.435,1952-03-15,false,32690456.3448,17:14:15.58 +-23160189,44049776490681224,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.435,1988-11-22,true,734175722.595,8:19:49.8 +1508857335,9679784280432078,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.435,1966-11-01,false,297390700.513,20:14:33.18 +248728157,57546347362784936,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.435,2009-02-21,false,284022311.769,13:26:57.45 +-348887483,17420252433300060,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.435,1967-04-19,false,1235493981.19,15:24:35.24 +1903051796,22644694356625512,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.436,2012-09-23,false,521528157.653,5:41:23.14 +-828573443,50333384127971736,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.436,1991-07-27,false,1036102975.67,15:23:10.30 +-1857628252,61024741444886144,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.436,1963-09-01,true,134551000.263,2:9:34.29 +-872145657,21459165657962148,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.436,1951-09-24,false,332456191.597,2:32:28.15 +1962023750,16263680825800654,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.436,1971-07-13,true,1175278510.36,3:45:51.25 +-1688205685,41073816751727528,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.436,2001-06-21,false,585093568.057,11:15:23.31 +2009686711,63494733705961736,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.436,2005-06-21,false,272495030.394,1:7:35.4 +-1803929285,83302904148973440,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.437,1968-05-15,false,580750841.231,3:26:14.19 +2014720381,265528919978424,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.437,1955-04-04,true,1210735456.92,16:15:37.47 +-1966501288,64449117853501376,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.437,2006-04-07,false,990141258.782,3:21:27.52 +-1011086184,13844937088295988,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.437,1955-05-22,false,129201653.834,13:19:1.52 +-735877111,13357312245203200,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.437,1956-09-25,false,50092085.7948,14:24:5.11 +1271023320,23815418286731428,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.437,1985-03-06,true,275498116.908,14:1:2.53 +1126664145,72572818451153344,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.437,1956-10-04,true,1154237304.93,3:47:53.47 +-1794269912,21135618948182764,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.437,1986-05-22,false,639294042.237,9:29:19.48 +1689294035,63035780541143144,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.438,1985-06-13,false,879765378.855,5:5:52.8 +1889036923,70570073316290776,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.438,2012-10-04,false,1268177670.55,4:27:49.46 +-698193217,5080186955011236,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.438,2012-02-04,true,774891541.649,11:48:57.48 +-3712599,53109309544784848,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.438,2007-08-21,false,1250645366.9,15:56:37.11 +2095676748,20440846107666588,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.438,1966-02-14,false,1333030379.5,5:15:34.36 +-772700572,80331739961112544,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.438,1974-03-23,true,583768933.61,3:6:6.24 +1779731330,72839331003390032,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.438,1995-02-05,true,1007880209.36,17:7:51.53 +-1111346276,26037096924161740,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.439,2006-08-16,true,364320204.62,4:56:51.58 +-2086728122,1777980716271780,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.439,1984-11-20,false,505513580.986,12:6:14.35 +656932175,92120058861962368,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.439,2010-05-08,true,1249778547.37,18:3:23.41 +-1420144702,54129204725586576,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.439,1970-01-08,false,1168425534.45,6:43:20.9 +1817946759,24247262909608980,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.439,1996-02-18,true,632659462.152,11:18:12.51 +159098432,81985762409667952,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.439,1960-06-26,false,287730109.495,17:54:33.49 +-2134307853,40122495254016584,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.439,1976-09-23,true,687059929.264,17:1:41.56 +-1368956806,11084009777671926,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.440,2008-10-08,false,1163815488.16,4:34:42.56 +-1781592541,81591552516404176,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.440,1988-02-24,true,970425046.153,9:3:43.1 +2099158505,85388948801580224,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.440,1969-03-15,false,1111438874.04,17:54:31.56 +-991164017,25971929558209976,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.440,1986-06-23,true,899519573.843,9:32:26.31 +-2089458048,39214154633558048,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.440,1958-08-13,false,1143144806.15,3:15:39.43 +-1893259297,70854654468536160,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.440,1987-04-24,true,1402575227.04,5:6:21.54 +1054469432,54777105480376288,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.440,1990-01-03,true,1240134746.39,2:14:9.4 +-138365339,20553189116381420,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.440,1970-01-10,true,711215767.861,19:48:55.12 +2022969543,769841070656593,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.441,1993-09-26,true,494017966.266,14:53:47.34 +1772155660,47299863564615160,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.441,1951-05-07,true,1251708438.85,18:34:33.51 +641337342,88281445202299760,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.441,1995-02-21,true,1145953815.05,11:32:7.16 +685049508,42534966639042192,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.441,1980-05-12,false,1277669622.35,21:27:43.6 +653396840,75800407021185552,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.441,1952-09-18,false,905839714.758,5:38:23.3 +-124984709,19411926505859892,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.441,1956-10-25,false,1228455929.1,4:8:49.16 +180624431,82423869740600400,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.441,1969-09-25,false,883762156.312,13:50:28.36 +-524107443,81733677384736896,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.441,2000-04-14,true,842261842.262,13:2:13.58 +676044810,91606785489502192,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.442,1998-05-27,true,763006020.585,19:10:16.49 +604308082,51071552129088096,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.442,1989-08-07,true,1295443958.64,7:24:4.57 +1611403391,87218977494536352,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.442,1999-09-01,false,1362797079.46,4:22:30.4 +546822790,87285960895532960,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.442,1954-06-08,false,549687685.996,21:36:9.23 +-1236093268,8705773892935844,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.442,1968-03-21,false,994391752.12,1:19:29.34 +-1571011966,48039990547253840,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.442,1963-05-20,false,88444619.2056,22:28:20.31 +-924655161,22723244906783212,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.442,2009-03-13,true,62740034.0701,15:12:22.57 +8877586,72192673789079200,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.443,1973-09-04,false,1251487783.01,16:6:12.32 +657291107,32417603256287212,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.443,1969-04-11,false,464275744.486,20:19:20.18 +801732842,54464932531895376,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.443,1955-07-06,true,950885546.517,6:42:55.30 +-147159960,80314548487535888,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.443,1968-09-15,true,988564827.006,13:40:24.39 +35276140,90382304740538624,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.443,1953-02-24,true,1289701898.44,8:20:19.14 +1801948347,35450149307886428,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.444,1950-10-01,false,504851424.745,2:47:7.25 +588727576,66877455969645792,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.444,1987-01-17,true,629903671.007,8:30:19.51 +-1944140294,82336745289219472,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.444,1991-05-04,false,439379168.246,19:13:15.35 +-1617635869,65765496947211552,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.444,1951-11-10,false,979976655.278,1:8:43.28 +2056994243,87843460154600064,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.444,2000-05-19,true,1358289553.18,4:7:12.48 +-702594626,74645125142865648,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.444,2003-02-27,true,1170028394.25,20:36:20.52 +-179602954,16364194524243108,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.445,1974-10-03,false,66822518.1094,18:10:52.12 +-263849993,76305975842793824,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.445,1976-07-22,false,513914850.567,13:31:57.29 +-74772031,80608578925800560,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.445,1987-01-15,false,926193449.995,8:40:30.31 +-199314368,73767183211059008,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.445,2006-07-06,false,641317302.886,14:27:8.7 +1234679601,90688249195119072,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.445,2005-03-10,true,1132832390.09,6:31:11.40 +1732857045,33785328379957044,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.445,2010-04-27,true,1363423256.84,8:27:22.45 +2003992798,76172189503108368,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.445,2012-07-04,false,1106031782.97,15:19:24.16 +1169883949,79386758861386832,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.446,1992-09-19,false,325968714.974,15:10:31.43 +-2033445252,21139678254917592,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.446,1954-08-10,true,1358130559.47,12:38:33.37 +-359049038,70620764165918216,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.446,1996-09-24,true,1375430800.23,12:54:49.23 +-897478498,33939451787431088,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.446,1986-06-21,true,1424843493.66,12:55:54.37 +102846342,41275003243569648,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.446,2009-10-16,false,329327613.684,5:53:41.44 +126124878,64595972415205416,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.446,1990-01-25,true,439742196.463,13:58:26.58 +-1560898674,76753063031630128,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.446,1998-08-07,false,1157435683.24,19:32:19.22 +-1260216933,29411421424535288,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.446,2009-07-10,true,533523805.307,13:4:36.19 +-1640680999,89848525955130656,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.447,1984-01-05,false,103993730.08,19:29:4.27 +947398941,44025301845253272,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.447,2003-04-25,false,363977202.518,2:18:42.30 +-1715977921,31899957102478624,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.447,1990-10-08,false,251117108.284,3:50:27.28 +1841197086,9248302543090360,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.447,1978-06-01,true,722856546.436,8:16:11.46 +175647443,89565773913371776,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.447,2001-10-01,false,275627898.027,6:20:19.44 +-2032004589,11745537748062280,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.447,2003-07-09,true,61465509.3884,9:47:29.38 +2029339369,71054282866115912,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.447,1987-07-16,true,1237143392.09,12:7:1.9 +638529863,73407082033041424,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.447,2000-05-08,true,901860051.865,18:34:17.47 +2116788267,85532645414663680,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.448,1983-01-26,false,1365005223.52,22:5:9.49 +-2035451168,73462515453913536,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.448,1973-09-03,false,1345343356.89,18:42:17.33 +1516535871,15213852568644034,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.448,1977-11-02,false,335444272.076,2:33:44.14 +1494977105,117664898233088,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.448,1956-03-13,true,1403892512.71,16:41:17.51 +-132610575,78947734043522096,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.448,1970-01-22,true,1208325308.23,22:25:52.41 +119210802,24927712759812220,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.448,1994-05-17,true,717074716.28,22:7:50.48 +-788220848,78106261335874064,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.448,1954-03-02,false,84833490.6732,17:36:21.54 +-1468405212,76275944904829584,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.449,1952-11-02,true,607673644.683,10:23:35.37 +1360768767,23978004463639768,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.449,1980-09-11,false,821689912.534,7:11:14.28 +509960805,34646340000377796,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.449,1999-04-09,false,461884846.756,7:44:36.37 +1733847805,33354562547548028,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.449,1969-03-03,true,84611885.6545,7:9:20.47 +607684605,52213890816370696,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.449,2002-02-17,false,1288008692.8,16:18:55.34 +1110318141,21585820169874444,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.449,1975-11-17,true,1320254141.35,5:14:46.43 +-749127835,64196722186291208,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.449,1977-02-06,false,269709659.381,11:11:55.56 +-540221846,4390277741039370,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.449,1952-09-04,true,1378139488.64,15:9:33.34 +1871816255,91360091723172608,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.450,1952-06-05,true,1045794131.59,8:16:1.56 +1884368499,11010169232156038,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.450,1970-01-16,false,687082816.324,13:41:26.24 +-372366528,51897098562403072,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.450,1998-01-05,false,428115999.071,9:23:5.40 +1432200751,69509663493516640,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.450,1961-08-24,false,860610757.844,4:1:45.48 +-1265478892,89632786717812832,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.450,1998-04-07,false,399793636.919,21:7:24.38 +-829450900,48405447029119096,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.450,1969-11-22,true,1126460580.0,16:43:58.14 +-360786890,89648607090294432,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.450,1969-03-12,true,34581843.3602,8:56:53.19 +-62049726,16566837659026114,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.451,1980-08-11,true,122580807.543,2:23:23.50 +-362225276,27990361296156968,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.451,1981-05-02,true,422517152.002,11:50:41.30 +-248991511,9227003722709422,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.451,1954-11-17,true,675414627.131,3:5:22.5 +-336673964,52294220994619056,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.451,1950-07-21,false,842245279.56,14:39:16.17 +-62812614,64066643895423168,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.451,1994-05-06,false,1111466362.57,6:22:10.27 +-1397427754,42742637548310368,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.451,1988-05-13,false,716903989.036,19:36:16.31 +1212662754,11367559515807754,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.451,1984-03-15,true,166919475.343,21:36:58.51 +-37473899,22779173329481996,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.451,1989-06-24,false,177858194.674,16:55:36.21 +-944069002,2655583856475668,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.452,1956-11-17,false,640988368.183,10:56:25.41 +177771981,17821790312224134,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.452,2008-03-24,true,425277320.879,7:41:27.1 +1062132997,44673071111753968,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.452,1958-11-05,true,1088116634.56,2:5:17.7 +1365253193,69072852745757936,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.452,1985-11-14,true,581061818.614,16:36:13.50 +-621807132,82500948189905600,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.452,1958-06-22,false,66015863.1216,13:23:3.29 +-2111043398,68409762728810336,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.452,1959-04-05,false,1352257928.25,22:20:5.14 +276704577,65129124768821712,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.452,1970-06-18,true,177689122.132,6:39:50.19 +-549136704,10596135283827662,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.453,2003-03-12,false,410412143.214,18:20:4.1 +-1483906919,10420813000200448,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.453,1970-01-24,false,697057024.774,1:12:21.16 +1020947982,5534397032912415,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.453,1987-09-04,false,131883359.32,7:48:55.42 +-1941045714,51943217670977824,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.453,1970-05-09,false,274053764.067,11:3:11.37 +118632673,72331177751248224,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.453,2006-05-04,false,706714298.452,18:10:58.51 +-303423574,29449718843959820,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.453,2003-04-08,false,301207230.245,14:8:45.50 +-809544262,27534078772406856,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.453,1964-09-23,false,1150617823.48,7:23:41.16 +-263107630,16020053912493680,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.454,1963-05-25,true,753115266.439,2:51:40.2 +1229994349,429717537212559,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.454,1959-01-09,true,1388631002.11,10:26:21.7 +487174779,37449223092714856,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.454,1956-09-08,false,333676376.7,12:10:42.10 +584781446,83392096700061888,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.454,1976-03-10,true,251166163.113,1:2:48.46 +1636758915,40235396956693312,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.454,1962-06-27,false,1055567046.23,18:12:40.45 +-1196560179,56741762553810984,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.454,2013-01-09,true,382436452.678,17:6:39.27 +-384669928,79462531425032160,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.454,1995-06-19,false,630226857.117,5:1:37.58 +-2134291165,21469286343072412,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.455,1985-11-12,false,120086693.848,13:16:17.50 +1121811909,55204564144871128,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.455,1952-11-07,true,914291130.084,8:26:36.9 +-1348121334,37473846443645512,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.455,1961-07-17,false,881776566.094,1:52:8.9 +-179749810,61142880523327528,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.455,1999-11-11,true,282686818.847,14:18:31.8 +1005497310,91467042997812416,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.455,1962-04-09,true,767480316.671,11:33:35.46 +-913117740,13850275911108168,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.455,1969-01-08,false,537631080.782,17:6:19.49 +-1882532166,20841421061219460,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.455,1967-09-23,false,607083677.048,20:46:6.34 +-48515049,37185543578140848,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.456,2009-11-06,false,79503493.0064,9:36:49.53 +1274473918,75913305746235728,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.456,1962-03-20,true,1059767138.7,4:22:19.56 +-1606048425,20510721867754252,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.456,1969-07-03,false,425370248.923,12:43:55.42 +-820742960,71366201433105032,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.456,2003-03-23,true,560693851.492,22:2:10.29 +-313798594,10285586825459866,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.456,1977-05-26,true,38913083.2857,6:35:44.55 +-1155688106,14143861323467110,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.456,1989-08-07,false,352512189.952,3:3:26.9 +1824180716,41457335069846608,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.456,1976-05-21,false,224612583.65,20:40:15.4 +444515729,63173277091537928,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.457,1950-03-11,true,1099886087.17,14:46:28.43 +-1488731107,13377870720810128,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.457,1960-03-13,true,341385771.976,17:23:5.33 +-1177402238,4867467928798116,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.457,1954-07-08,false,533017029.226,1:32:26.55 +1960484493,17606665600693064,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.457,1955-10-18,true,123036929.951,1:19:50.14 +-719706889,7058676148050289,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.457,2002-09-01,false,214628884.788,4:53:45.23 +-1300119386,88192293022378688,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.457,1953-04-19,true,566061308.781,14:30:13.40 +-5043217,76073343349063360,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.457,2013-05-11,false,1014561886.43,21:27:16.45 +338704066,52378102278188016,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.458,1968-06-26,false,800982301.808,21:37:37.17 +685116079,3569272553345700,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.458,1978-11-15,false,939783821.682,21:17:42.44 +-1569178253,4454222026090486,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.458,1957-07-26,false,1137563692.18,7:19:48.34 +-241326518,76214726186004560,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.458,1961-11-05,false,1325248338.95,17:54:58.58 +-1067941491,8761946368352010,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.458,2001-01-12,true,463574399.201,14:34:33.17 +-2066703356,52924584813928312,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.458,1993-01-18,false,33500082.9166,15:23:31.10 +515810678,40540393638520728,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.458,1962-07-02,false,787204070.682,18:38:23.14 +-847262057,63762606752645584,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.459,2001-08-11,false,1333982965.63,19:20:57.26 +2143413362,25605843193622028,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.459,1969-11-21,true,1005900740.78,5:1:25.50 +-1017630928,59968234923802752,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.459,1994-01-23,false,1382615058.31,11:6:47.40 +-524657848,37273036740613816,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.459,1963-10-03,true,836488838.971,19:22:37.29 +196182972,6678633622872341,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.459,1992-07-26,false,274564454.858,8:44:26.10 +31324065,1237325541104414,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.459,1966-03-02,true,604060162.604,6:50:37.52 +-1618559596,90561395796810432,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.460,1964-02-11,true,1081710123.54,3:24:28.41 +-1164444702,12047210639343156,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.460,1977-05-07,true,1395810472.4,19:42:13.23 +-69760934,26113631734460600,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.460,1985-05-05,false,282523496.948,4:5:25.9 +111122250,10053309503534736,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.460,2001-08-22,false,1432642299.73,7:50:26.41 +1714321682,7496296128598139,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.461,2013-06-22,true,1103422306.31,15:50:22.29 +795037925,85152205144001376,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.461,1970-04-07,false,556539156.418,11:49:19.51 +-1313394742,14972683983199642,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.461,1999-01-11,false,460617674.277,13:46:27.22 +-971195278,18987087513188456,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.461,2007-08-16,false,283371533.621,22:32:44.9 +-301282392,32339465174824336,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.461,1987-11-22,false,580942232.045,4:31:26.11 +543834753,3038774037364715,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.461,1984-01-21,true,1357889694.7,9:58:47.12 +1887570685,1588702430482073,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.461,1977-08-04,false,361400248.363,11:7:53.55 +-837868764,57602748729516216,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.461,1997-10-18,false,34092611.8589,11:31:4.22 +-829623241,10010131656881244,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.462,2009-06-07,false,309137874.194,12:39:29.48 +1215386804,62715387082840192,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.462,1972-01-13,true,1329723525.93,5:40:58.52 +361822376,47070624193242280,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.462,1999-07-15,true,421754268.278,5:29:37.55 +-2085665245,27351190202659512,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.462,1966-10-27,false,955788763.238,7:19:32.45 +-236694109,64607287525859208,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.462,1951-06-02,false,440500516.631,6:50:44.33 +741528374,69760728980981192,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.462,1954-06-24,true,1280844683.47,7:27:17.20 +1149456675,68212721177789072,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.462,2007-02-04,false,1036497226.34,12:21:47.6 +-1964523583,64906408913000512,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.462,1956-07-13,true,991987584.176,17:51:21.26 +1562442890,79379173393536896,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.462,1988-02-21,false,1093172967.76,18:13:10.34 +79873295,27099771303482872,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.463,1965-10-27,false,1416517801.72,11:48:1.51 +-476837690,30285967885701532,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.463,1963-09-22,true,281406806.064,11:46:20.37 +1519193160,35265240481526384,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.463,2004-05-12,true,1351411779.94,17:35:24.30 +1530735962,71841743436681616,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.463,1955-08-08,true,1262764235.63,19:58:58.14 +388984198,70989425930514840,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.463,1975-05-27,true,909903848.618,6:7:16.26 +3015799,84802653073767456,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.463,1965-02-13,true,1023300130.83,3:43:57.54 +1870123703,2661499224190576,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.463,1981-01-24,false,373956550.403,4:35:41.11 +1661108985,30323775648076064,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.463,1989-06-16,true,735482776.646,20:26:1.27 +1311592824,37709005878335304,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.464,1961-08-17,false,627874502.976,16:22:23.21 +985719360,23551242783879344,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.464,2000-04-05,true,31911548.1527,2:4:35.4 +-457318730,69519551694498408,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.464,1956-11-25,true,548135389.311,6:35:56.53 +-1706616667,27527096419380052,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.464,1990-10-08,false,791347943.749,20:17:4.47 +1501166641,91666184113529360,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.464,1999-05-07,false,1208019196.91,2:26:31.57 +1382088581,27620977160441960,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.464,1994-02-15,true,399470298.685,11:10:47.37 +54287432,24168561803954864,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.464,1992-03-01,true,550854555.812,21:50:17.22 +1432334264,41555122749341408,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.464,1963-02-21,false,473683613.224,5:8:10.12 +1483445844,77634860006184608,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.465,1999-03-03,true,965530405.352,11:50:2.36 +555994010,16801487471943036,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.465,1989-07-14,true,70873403.3622,7:9:27.2 +-500610719,70751463589926112,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.465,2010-06-15,false,728292402.006,17:16:58.13 +1891494538,35167979947987908,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.465,2004-01-02,true,419325545.13,21:18:29.22 +150457986,13995491857058314,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.465,1980-04-01,false,1357897614.31,13:25:14.53 +-1657890076,86331931608666288,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.465,1959-03-21,false,895599221.164,21:53:5.25 +1323870478,76197474424107536,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.465,1985-10-20,true,440977075.905,2:39:47.22 +162282764,85615510141742160,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.465,2003-03-07,true,735332698.155,8:8:41.58 +40698233,81606658152788512,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.466,1997-06-17,false,414904236.337,12:34:42.7 +-1452472759,80845118157286816,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.466,1993-05-16,false,499262210.406,20:7:55.53 +1703436301,19527366868989052,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.466,2001-05-01,false,1051662781.54,12:47:5.40 +1273694197,65760961005605144,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.466,1976-09-15,true,961194507.287,1:8:5.41 +-942661941,70663808398181224,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.466,1981-08-26,false,1334600589.5,13:56:32.40 +1222077352,30472389118735872,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.466,2002-04-13,false,823789815.96,9:55:44.4 +970254847,18736110130911192,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.466,2011-02-27,true,258593526.191,18:16:28.51 +575504696,49435792280153680,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.466,1975-04-21,true,978711538.535,11:19:32.58 +-791617416,80264455782186832,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.467,2010-10-06,true,21198012.8689,12:34:13.6 +-2047914845,49375045667830120,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.467,1983-08-16,true,552986066.598,9:53:6.55 +-1850671936,41292374637923360,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.467,1998-06-03,false,1161637805.77,18:7:53.41 +1003375436,58428944569781360,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.467,1963-08-03,true,67983342.8513,22:35:39.2 +1705617374,83457530612907072,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.467,1979-02-21,false,176000237.239,14:33:47.32 +-822417733,13079444753784270,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.467,2005-03-15,false,997448422.487,9:36:56.15 +-805469280,4896894991592039,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.467,1960-03-17,false,1406431124.67,21:33:45.5 +1408747298,87481198534112656,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.467,1963-06-27,false,41931239.3494,3:24:22.10 +-1950497972,22790117188590932,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.468,1967-10-08,true,790616613.348,8:18:44.8 +-1223616111,7250423050930381,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.468,1954-09-04,false,74727317.2696,21:24:22.9 +-1390813917,30158112352893616,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.468,1987-07-14,true,783595414.15,5:29:21.39 +-520293818,53954620316707128,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.468,1999-06-01,true,1051152309.79,14:49:9.53 +-230115455,86375171222783104,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.468,1993-11-06,true,433457423.292,20:24:46.41 +1872184346,21576997899370908,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.468,1983-08-04,false,1334479059.4,6:40:6.5 +-2011101000,19616382969927424,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.468,2013-06-04,false,723775583.829,2:13:45.42 +-44719257,5453825672696238,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.469,1985-04-10,false,1330320569.74,18:27:16.26 +-844016038,65240569526320696,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.469,1978-10-19,false,985496973.446,15:38:33.41 +871380653,27829873458658048,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.469,1988-05-01,false,617591927.047,18:49:4.40 +1637330818,88389130275505296,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.469,1952-02-09,false,160237765.312,19:15:11.30 +119255855,15416274308387266,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.469,2013-03-24,false,1010776226.26,5:24:38.57 +521563992,12515793809395896,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.469,1955-08-03,true,339866375.686,2:47:36.29 +330344766,81976013763998368,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.469,2007-05-23,false,875270217.107,17:56:55.12 +-1495083760,3201087199357245,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.469,1962-07-13,true,20365352.553,4:11:29.51 +2122690870,12302983886180844,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.470,1972-01-09,true,269186185.642,14:4:4.8 +1762329999,88671855376119296,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.470,1956-11-03,false,42893721.2974,17:36:58.41 +953429856,4034195253490504,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.470,1957-02-23,false,1273273864.05,13:12:13.24 +-642320878,17720763763506228,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.470,1962-08-09,false,1221850039.05,12:16:10.39 +2073765145,59737294420964080,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.470,1954-02-27,true,240454761.099,11:27:52.49 +-1426479647,29191965429520928,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.470,2008-09-06,true,1394707366.79,22:39:28.52 +-628618881,41053504981027680,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.470,1966-02-13,true,188221551.045,16:18:23.37 +-1972756340,23445923549043868,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.471,1969-05-01,false,721882684.933,5:7:18.51 +-1466657706,50428900675500656,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.471,1951-04-09,true,209751622.493,4:56:55.56 +958395777,12377164372799590,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.471,2012-06-19,false,739424838.826,3:5:50.52 +728282161,23245981506447236,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.471,2005-06-21,false,1240287722.55,1:12:16.4 +-997078414,38491554006808072,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.471,1977-08-02,true,985860039.563,17:16:11.33 +-1061877496,24911021450323704,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.471,1969-05-17,true,435720429.245,22:52:44.54 +312062745,65377043745080768,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.471,2010-07-26,true,685658789.48,17:28:39.3 +48132521,55600565936117904,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.472,1991-01-11,false,632079688.264,9:10:54.46 +1715173535,25947155317538408,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.472,1969-03-24,false,248841911.407,2:22:16.52 +-1462369755,87211047118593120,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.472,1954-09-25,false,641071334.695,22:33:17.40 +-1983337223,26917141033195212,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.472,1987-10-19,true,96615590.3537,12:32:56.5 +2047408994,20686388532198012,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.472,2000-03-24,true,76830453.6362,18:25:15.42 +-1982712714,409573684178565,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.472,1956-09-20,true,928895906.263,3:38:36.32 +-376277227,27605402159377736,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.472,1964-10-18,true,327176169.724,9:17:55.13 +564202353,85887622223340144,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.473,1998-03-25,false,340412251.083,17:23:10.27 +-849106974,84864263648514752,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.473,1997-01-03,true,873703785.957,6:36:44.17 +1154991847,4809818838134846,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.473,1972-02-27,false,222193205.481,9:28:23.51 +-1219798226,31013807498447116,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.473,1973-04-06,true,1086524843.15,9:33:36.5 +1060661119,6480032071549276,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.473,1974-02-12,false,736228322.689,17:40:35.18 +1573676221,57842682049154672,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.473,1954-06-03,false,474381776.309,18:1:52.50 +1933482834,61646001471093176,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.473,1986-10-05,false,503678960.126,17:28:40.40 +1327039894,79248978910612640,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.474,1990-05-01,false,436149283.371,8:29:6.22 +-1233618332,9167511668576716,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.474,1984-10-10,false,269786550.63,9:7:35.42 +-1559458420,68985190430727736,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.474,2008-10-08,false,100943244.115,20:44:47.5 +137715099,75859467472230304,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.474,1961-02-15,false,1220715769.64,10:52:21.14 +-1846545592,72244463000386400,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.474,1951-09-17,false,982663828.859,4:21:1.15 +1218446035,70870478064083664,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.474,2001-05-08,true,53162362.4767,2:40:45.25 +499850656,9155799202681518,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.474,1963-04-26,true,1011075923.5,21:47:13.57 +-1389139026,38742949034760800,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.474,2005-05-27,false,1295933772.69,10:3:52.48 +1221170648,86359746145210304,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.475,1987-11-11,true,120905406.384,13:24:44.54 +-1885704050,49086387632812296,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.475,2004-04-20,true,993149668.717,8:38:32.46 +1215587966,48742629830813008,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.475,2013-11-11,true,967456089.697,18:7:54.6 +804416374,71512602715155704,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.475,1983-09-18,true,362355690.505,10:34:20.6 +-1671927114,29427945899645728,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.475,2012-03-17,true,242523109.267,22:4:29.23 +260115818,68313139482486040,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.475,1968-04-06,true,27079291.4716,18:32:55.13 +132706751,49678739319437976,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.476,1968-05-26,false,301730599.834,14:23:19.6 +-879849556,64401258434736520,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.476,1988-06-21,false,1335441187.68,19:29:33.32 +-755424572,13710879511982726,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.476,1968-04-07,false,1240441857.37,17:42:8.44 +-1545727688,29934683705033904,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.476,1973-06-22,false,326615268.063,8:2:8.35 +-695198535,80587072388901552,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.476,2010-02-13,false,1291602516.17,5:51:24.57 +-465154828,3823252524039823,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.477,1958-09-08,true,1200045112.92,22:25:16.12 +-1810126310,73552196215243520,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.477,1975-10-06,false,280068975.774,18:37:52.40 +619013244,7354245157770056,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.477,1974-02-25,false,1368311229.66,4:10:35.7 +-792853304,52132828859326032,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.477,1963-02-21,false,967974108.403,22:28:3.28 +998252559,53527887482903216,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.477,2001-02-26,false,183434514.494,20:32:43.36 +-770316205,2872439069076173,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.477,1993-10-19,true,1349909720.76,7:1:27.41 +770047501,72538066538259872,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.478,1958-08-13,true,448768558.217,19:7:7.50 +-338414139,28298853732945632,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.478,1959-02-10,false,983229960.339,14:27:35.42 +-1648625389,28971084127756780,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.478,1959-04-15,false,893738884.715,14:3:38.13 +837145858,18163628982325536,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.478,1966-05-18,false,480640913.179,8:52:39.53 +1894639201,80314465826563008,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.478,1992-11-27,true,654011159.67,9:57:39.28 +-303360590,55693767928110832,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.478,1970-11-09,false,674470222.388,2:36:33.34 +439208081,29517518130319792,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.478,1994-07-01,true,681610529.704,19:23:26.1 +-931859180,23978234072733500,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.478,2005-08-25,false,762481315.863,8:11:48.9 +1161688760,29108685574447412,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.479,1993-02-10,false,441284442.171,8:35:37.47 +1825002732,64844112083087832,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.479,1973-05-06,true,685672287.992,20:29:43.2 +-1483422988,79294492614922912,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.479,1958-10-08,false,823730970.695,14:28:17.15 +-876466165,53619419617568392,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.479,2007-10-26,false,499841355.048,7:43:10.19 +733579789,57166167224591616,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.479,1995-08-26,true,864491205.503,18:46:18.31 +-1124618901,27870714613372180,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.479,2010-05-14,true,1189184241.71,5:8:49.14 +-1174112769,19021417536920944,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.479,1973-08-23,true,468028729.658,10:42:29.29 +-78419602,27870577797534760,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.480,1962-08-03,true,8420928.20856,9:6:40.31 +1367399499,12187267145522986,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.480,1994-09-17,false,479268362.878,12:14:29.41 +-888804930,88681481836037680,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.480,1957-01-19,false,775225256.97,14:48:15.35 +-531617407,20074495422055700,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.480,1980-01-18,true,772683730.032,10:54:4.24 +1182277869,70497466611185744,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.480,1956-04-19,false,1381378057.72,19:7:38.46 +796291638,52197517781938496,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.480,1952-03-22,false,1361773102.6,12:8:37.38 +1697965368,68838724866544048,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.480,1961-11-10,false,384815630.764,1:20:23.27 +-218564311,79831782994420448,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.480,2000-05-17,true,997398935.7,17:31:36.24 +80154700,39216045082925496,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.481,1959-04-25,false,532966746.65,3:11:21.38 +-1763491424,17881103994740552,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.481,1971-07-25,true,1313622357.99,5:3:58.18 +329363238,10125930226554276,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.481,1993-08-17,true,246840490.314,7:3:38.57 +-1983758666,90154618881713376,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.481,1950-08-13,true,85315932.3666,3:25:55.22 +631609361,3071991886734950,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.481,1983-08-24,false,1227072899.1,15:3:41.11 +-1064391337,19230989859818140,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.481,1954-03-07,true,168648747.633,20:17:34.36 +-1323338114,11799419911471626,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.481,1975-10-14,true,1215526657.22,7:9:40.34 +-1251160541,32273465410018460,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.481,1976-04-21,true,791040367.264,4:16:48.23 +-1797404210,51378915523271272,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.482,2008-10-07,false,1331565159.98,16:8:47.10 +-300222587,66812242264756448,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.482,1974-02-08,true,344431869.004,16:55:20.16 +1625609916,63225358524711808,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.482,2008-02-19,false,1098330018.55,18:55:14.22 +-1056262091,61458666821764568,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.482,1953-07-25,false,1103823495.0,13:42:53.9 +560359325,47554105918642496,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.482,1991-05-26,true,611084153.559,9:13:16.35 +-395632970,91441198397411008,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.482,1979-06-27,true,646965643.54,8:58:39.53 +-1384790449,67516671368869480,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.482,1981-01-09,true,1378085705.29,20:53:12.40 +-2093439743,26221376455596912,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.483,1974-03-23,true,517297486.762,17:13:54.51 +-504921389,39054900669420912,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.483,1973-03-08,false,1080214383.41,13:26:3.49 +119560257,32097906190614960,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.483,1974-07-09,true,276839995.327,19:16:44.38 +1591008623,41105377580192000,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.483,1956-11-08,false,689149100.908,8:21:47.10 +-903333457,45139712345309064,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.483,1977-03-16,false,846410243.366,3:17:55.8 +-777716506,43973290814321368,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.483,1988-05-19,false,1087265458.08,14:41:10.25 +1643747693,250748684755957,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.483,1971-05-04,false,642326937.573,18:30:36.26 +-673973612,57457148514594160,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.483,1970-03-17,false,707989179.818,14:31:23.42 +-1235938759,31320065121887528,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.484,1964-10-10,false,1271221871.34,10:18:56.33 +1716469422,17950061788407502,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.484,1965-10-20,true,1278819202.14,9:4:3.47 +256671069,8677555180445737,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.484,2011-03-22,false,1058123053.99,17:52:44.35 +1896004538,81665465011335824,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.484,1998-10-17,false,917024868.66,4:12:9.26 +871523713,91408066962808944,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.484,2003-07-09,true,926436975.394,12:30:37.35 +750862519,69611039817912144,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.484,1964-04-03,false,397030657.67,17:10:43.49 +1101678074,83341851028118240,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.484,1971-07-10,false,468417875.339,9:4:28.23 +-1583182434,21643783621513164,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.485,2001-03-01,true,241563372.873,15:41:56.32 +-1890543074,84700762938433616,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.485,1975-06-27,true,121132242.579,6:20:4.48 +1410397165,20270083467390824,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.485,1963-07-02,false,153353413.092,6:17:18.18 +1579375293,37234594164193984,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.485,2003-11-25,false,1073231464.12,18:27:19.21 +374894996,16726403825328692,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.485,1998-07-10,false,954014823.003,1:40:1.40 +2120500817,43639265164772120,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.485,1983-07-01,true,566594589.061,4:45:48.42 +-732048910,37979634963742608,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.485,1987-08-19,true,285502125.169,20:19:27.48 +-667158477,59384761247241616,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.486,1974-07-18,true,987083205.15,14:27:49.13 +1911642878,51595239454777624,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.486,2010-02-11,true,3722763.83589,6:26:1.37 +1179712721,11079358224034356,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.486,1994-07-07,false,603887862.436,15:10:44.20 +-1996167029,65963073639343616,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.486,1952-08-09,true,241648540.566,21:3:18.24 +72359387,1092752420595722,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.486,1974-09-15,true,29857121.9961,5:43:26.5 +448424852,90354758735747120,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.486,2010-09-18,true,982700853.821,16:27:30.20 +628801346,9504832632642560,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.486,1964-01-19,false,1245930997.64,9:8:49.27 +-1227647380,8245069156674816,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.487,1971-07-20,true,1235026018.91,2:2:40.57 +439602272,7908647956262390,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.487,1982-04-09,false,600680893.711,6:41:5.24 +843054211,64606321809127496,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.487,2005-03-26,false,707533845.273,15:37:4.20 +1173136363,76927185739874384,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.487,1989-08-23,true,855597440.633,18:22:31.40 +-1655838038,56121729528680400,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.487,1999-11-19,false,821628853.619,14:54:47.28 +214799771,59872852045944384,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.487,1953-02-01,false,1004936589.78,16:6:10.34 +-237146469,60985686131942848,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.487,1957-01-17,true,851383674.987,8:52:6.51 +949984918,86617549137929904,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.488,1955-03-03,true,1372788478.23,17:2:2.16 +1042469046,70512480805298088,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.488,1980-07-14,true,1032470022.06,18:10:6.21 +-1172498669,82843632519265616,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.488,1998-10-17,false,1381267514.77,11:38:3.38 +1647145752,3550545440876728,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.488,1981-03-06,false,867455864.06,7:22:40.43 +-1838069723,65074542678064320,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.488,1971-01-12,false,1328131652.46,2:20:58.51 +1733562683,54687986890468544,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.488,2003-08-02,true,947000471.584,9:37:32.56 +-1740452788,68762611671746896,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.488,2013-08-25,false,1416318281.69,5:28:6.25 +51415186,52620104400822456,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.489,2011-05-08,true,588876998.763,3:39:12.1 +-365275845,92052496947238048,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.489,1982-01-17,true,906963830.061,17:15:56.51 +-486185483,26905131758524640,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.489,1964-04-27,true,1292031341.14,1:52:37.40 +1979005053,50935126175875496,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.489,2013-02-07,false,692255248.952,14:43:13.35 +1957398772,81953852387959472,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.489,1984-06-25,true,560751795.526,22:53:46.8 +1398812623,4625811131411999,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.489,1960-08-05,false,779969801.504,6:52:16.49 +1332802275,90369602478811744,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.489,2003-11-03,false,257611548.092,19:24:32.17 +-2079044294,43789143276584696,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.490,1987-09-04,true,1285967356.2,2:50:22.14 +-1493690872,87846795150713616,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.490,2012-10-16,false,1245158650.71,1:19:10.6 +630976260,14269447648732416,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.490,2013-06-21,true,28844008.1879,15:48:48.15 +1389582067,11113470024988958,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.490,1983-08-02,true,1300554769.85,5:9:8.8 +-660371431,11325919015735686,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.490,1969-07-03,true,1075999279.45,8:32:20.4 +1407516674,6645955711205192,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.490,1951-01-12,true,1209043926.95,12:54:57.1 +-612236169,5752698495124091,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.490,1986-10-14,false,612722957.971,12:34:48.5 +1917464139,37034115597918616,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.491,2000-09-01,false,540811747.639,11:27:51.54 +-326028072,47027073369545800,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.491,1951-08-11,true,756330654.583,4:2:21.22 +1547786991,70408963334076632,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.491,1956-02-04,true,35420790.3757,6:24:56.37 +1974990438,45494834097414176,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.491,1972-11-20,true,1331128031.63,9:50:10.4 +680963030,14448909826707056,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.491,1972-05-14,true,1133289872.91,22:33:32.2 +-2058769919,54291297158323424,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.491,1970-08-25,true,1103587543.91,21:43:3.49 +438447910,42553555963237656,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.491,1973-10-25,true,1265728555.56,8:10:9.14 +835240521,26178272275060584,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.492,1978-01-20,true,318653729.883,1:46:44.22 +260363323,67423463288870392,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.492,1972-03-18,true,142538912.092,5:2:6.26 +267857958,72501801874193712,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.492,1952-02-25,false,392743913.166,20:55:40.19 +-1998213891,10194602529866794,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.492,1977-04-10,true,1245938500.67,7:17:56.41 +1592722759,63425652722710592,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.492,1964-11-17,true,54954990.9143,21:9:42.13 +-2050051359,44275552590326272,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.492,2010-03-22,true,625128923.895,13:4:8.31 +-830726048,34565723295646456,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.492,1966-11-26,false,653705488.923,16:34:25.14 +158619794,15858196752064728,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.493,1950-02-02,true,1324959235.15,22:39:34.15 +781536327,43445356745352160,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.493,2009-06-09,true,594603667.185,10:26:42.10 +-1724636431,15639343370706074,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.493,2008-09-19,true,270617594.989,15:31:23.38 +766508943,88086156132207408,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.493,1967-09-02,false,548101820.233,3:15:54.7 +-1091715726,19969666343857236,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.493,1963-11-21,false,494667550.294,14:39:41.11 +1827219693,75301667040163104,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.494,1955-03-02,false,526441459.047,19:26:50.21 +1724918916,88376437592769296,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.494,1954-03-24,true,873692733.681,21:5:53.19 +-1284492679,71417639734269664,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.494,1994-06-16,true,722730271.87,4:8:55.13 +-1011232277,46503341620125008,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.494,1990-03-15,false,372332202.429,8:4:58.32 +626807237,89749604453336736,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.494,2012-11-10,false,651361772.585,5:38:58.43 +224565948,82279237207659120,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.494,1992-02-09,false,208731344.34,7:53:7.45 +-1150272755,45144904016673544,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.495,1964-01-10,false,824480832.837,21:46:17.2 +-1317427780,11937484795351624,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.495,2001-05-21,true,148807088.6,9:55:21.9 +-1244974529,90265130709041952,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.495,2008-08-16,true,429563438.469,17:36:45.53 +-1544829513,28346083290240788,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.495,1961-03-19,true,1293675820.08,8:7:57.49 +2034544347,62283370321046752,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.495,2000-05-05,false,254438348.953,19:56:2.3 +2086408277,18075900359115860,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.495,1967-10-14,true,1330218766.41,13:50:27.40 +-979140012,81263624214212656,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.496,1989-02-05,false,1062053299.67,5:20:10.34 +-1955783365,23623532170192416,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.496,1996-04-08,false,1017685135.08,19:41:34.40 +1977357803,49870677109240312,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.496,1966-11-07,false,193090385.111,17:20:7.14 +-975351556,56302709269949544,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.496,1963-07-01,false,493649678.871,17:46:35.12 +1356017385,74189442007316672,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.496,1977-06-22,false,992676042.673,17:46:22.24 +87678500,27815234945689776,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.496,1967-04-06,true,688799060.241,17:54:55.50 +383832408,57965358132307392,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.496,1977-09-10,true,1237586744.49,8:14:6.21 +-682682680,83779682104176768,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.496,1991-07-27,false,1058083597.95,16:56:32.22 +1032493805,86031370764187520,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.497,1962-10-16,false,1288308744.66,18:52:17.47 +-89277499,64867542079099536,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.497,1956-10-26,true,101468271.5,10:5:27.26 +627117803,24773853625269832,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.497,1998-09-09,true,97671133.4025,3:2:44.52 +-1238743071,40486069525681728,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.497,1954-03-02,true,107635055.269,1:27:54.8 +1446247258,88523880650719632,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.497,1950-03-10,true,1287997856.25,20:14:54.12 +769329965,88228862856120368,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.497,2000-11-16,true,781013765.688,16:9:30.1 +-1680391052,71521567655634728,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.497,1981-03-15,true,459706076.558,2:4:37.40 +1995737961,17880886155852616,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.497,1967-07-23,true,273419883.026,18:57:45.15 +-968589119,77727373429721952,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.498,1963-06-14,false,168475540.334,10:54:33.43 +-1966704258,82131518710157472,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.498,2012-10-25,true,1002782324.02,13:4:31.46 +-1274670431,20797071338594264,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.498,1974-07-24,true,604684049.154,4:26:29.2 +-1081280787,14831761428506972,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.498,2010-05-25,false,275034064.975,13:10:23.41 +305342657,27548080253257604,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.498,1966-01-26,false,221197385.859,16:14:10.3 +582809361,24271896964039180,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.498,1985-06-23,true,866236533.91,19:13:7.14 +-922435684,3674580047328379,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.498,2005-08-13,false,762490150.688,22:7:47.49 +-1917468098,79425130183732144,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.499,1964-10-20,true,778893167.8,7:52:37.27 +-417884783,43561191866502840,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.499,1966-02-03,true,333802313.115,15:38:33.7 +-1968936149,38043761996060352,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.499,1971-11-03,false,1093618583.74,18:6:12.41 +299241867,22548061581728964,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.499,1975-09-12,false,1238510461.12,13:15:52.49 +-794846461,83796052590337216,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.499,1969-01-08,false,1195519074.82,19:17:24.4 +1616167362,16356330666769480,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.499,2007-04-25,false,1018001782.4,2:18:16.8 +857813733,14897911971831962,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.499,1988-06-03,true,773252231.43,13:45:52.28 +1906972729,34415895230202912,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.499,1993-07-06,false,962468130.663,21:12:19.17 +-673796713,81807881312654960,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.500,1975-03-10,false,1243119231.56,1:44:13.32 +-664952075,62839389821642176,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.500,2004-08-13,false,63656958.3881,16:2:10.13 +301649851,24654529087541820,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.500,2013-08-06,true,687174521.827,22:2:4.42 +2001666869,12255271966509896,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.500,1959-03-07,false,739693598.509,22:40:43.26 +885045621,78584621211049040,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.500,1965-08-22,true,390032942.498,8:12:58.31 +-2076953236,1940008216344197,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.500,2008-05-09,true,393805470.858,15:21:26.31 +-543625701,79211087904698928,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.500,1990-05-08,false,898216223.094,20:19:21.6 +-659288451,27806252908863180,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.501,2006-02-07,false,1244574989.31,8:1:34.24 +-675387449,3260523567617904,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.501,1984-04-25,true,74245047.3112,1:45:52.36 +911282348,8729548556097956,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.501,1987-06-24,false,1052538876.95,22:50:30.37 +376198705,52863082864315728,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.501,2004-09-09,false,1228754944.73,21:21:56.49 +-1177591463,66051992049969984,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.501,1981-01-05,true,759755015.979,2:38:34.55 +25416642,32665389904287132,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.501,2002-08-27,true,102032094.383,7:23:16.17 +-1856883670,64826214577039896,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.501,1992-05-17,true,1396771926.38,19:14:15.40 +444146391,17537006007151340,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.502,1969-03-01,true,567691986.128,16:8:5.20 +-31992226,77570036213161472,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.502,1993-06-12,true,976752074.123,2:43:12.27 +135214778,69398673192107664,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.502,1952-04-09,true,802124617.836,13:14:54.4 +-1617758873,43777638961880144,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.502,1984-01-05,false,188590498.816,4:32:15.17 +-1229079594,86203303852392240,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.502,1975-05-04,false,224873536.172,11:25:12.47 +145348018,84269284226409920,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.502,2011-09-21,false,1068934365.52,2:30:38.13 +-1800457812,21769087775412736,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.502,1970-11-25,true,864605086.856,11:49:19.30 +769309046,8574031005250570,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.503,1984-06-13,true,784912954.237,10:46:48.13 +1638106510,28950495703131340,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.503,2004-01-26,true,360149685.856,5:43:28.25 +-1164649092,45643950145506656,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.503,1972-06-11,false,790087374.987,8:37:15.26 +141194994,36225035159198888,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.503,1957-01-20,true,1017189526.96,12:3:47.49 +1162617040,65514012533543296,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.503,1995-08-10,true,1002392325.96,3:45:7.29 +675671394,54002655918557800,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.503,2011-04-23,false,1285585529.57,15:15:47.37 +-917762675,74364715224245312,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.503,2006-10-21,true,564696723.469,6:37:25.16 +587057780,1204771862111989,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.504,1991-02-18,true,558824892.683,6:58:24.15 +1038397851,4286090829497661,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.504,1982-04-06,true,783304248.262,1:12:32.27 +1184049586,42115548611643680,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.504,1978-02-12,true,249449733.204,2:33:19.41 +2096303165,62676613292545792,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.504,2008-01-01,true,1151114434.25,9:48:4.24 +-1021678658,55806774329521728,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.504,1988-04-23,true,44870214.864,16:41:6.39 +394194934,14141418772644240,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.504,2004-01-22,true,1279907429.15,1:47:14.12 +-2087027370,12587518453915854,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.504,1953-10-26,false,1008419097.71,10:38:45.51 +-1324869384,1067708931783352,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.505,1971-04-07,false,129500228.432,17:48:4.2 +-1387954476,25221477908819660,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.505,1972-05-07,false,142879456.738,2:33:32.42 +-1639885658,1767859393927434,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.505,1972-11-12,false,222037211.514,19:40:40.32 +-1956726708,64677097403096200,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.505,2009-05-12,false,663451290.544,18:12:13.35 +1713191992,49198374995778000,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.505,2005-04-21,false,543343824.953,4:33:22.53 +653492995,23889448779085468,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.505,1954-07-23,false,516339849.51,14:3:9.41 +-848896069,68548922571890760,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.505,1990-09-24,false,1384349047.87,12:13:20.8 +-1225582049,75294287131729376,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.506,1974-03-18,true,598910307.409,2:47:45.21 +-141380263,27422174098034576,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.506,1984-08-16,true,698805946.984,18:17:32.25 +201657894,34627727694612408,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.506,1975-09-07,false,422273108.069,8:34:34.37 +894246626,79802178513930208,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.506,1983-04-07,false,799216050.577,8:47:27.10 +290446074,74004325940579888,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.506,1986-04-09,true,1101379569.41,19:41:38.25 +1124250608,63986505926219200,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.506,1971-02-27,true,1263612481.28,14:40:20.56 +-1662349530,75474048251252848,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.506,1953-04-07,false,519222795.986,2:9:23.43 +-447320354,32491088511491092,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.507,1969-06-14,true,644804030.445,1:32:51.24 +-1660794313,45807481532858264,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.507,1988-11-11,false,799804611.323,8:11:37.24 +-1328852219,15621279730895104,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.507,1962-04-16,false,807169908.537,1:52:45.41 +2048620066,39601592998517160,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.507,2007-08-15,true,163384849.092,7:8:34.52 +-804584355,12097543851313090,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.507,1961-01-25,true,257505218.423,5:52:6.47 +1911543405,50944732030841808,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.507,1989-01-15,false,1333347597.76,12:26:16.6 +-1674193754,28364247267402600,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.507,1986-06-05,false,458197790.494,21:14:38.48 +1678720278,52788312075497240,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.508,1966-04-12,false,700616567.43,17:7:36.5 +-184766917,27802774748223876,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.508,2004-09-15,true,644358575.208,11:6:26.8 +1049183079,24385489084502056,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.508,2012-10-03,false,1263197718.34,14:14:36.29 +374048566,68353899252468320,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.508,1993-08-03,true,892993705.294,8:36:17.27 +1918299872,7285619920714834,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.508,1967-01-23,false,362848033.408,5:15:16.58 +567494213,32528305858697696,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.508,1995-10-23,true,849815969.334,13:38:16.25 +575274332,66614861152530072,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.508,2012-05-03,false,1083364589.88,7:55:4.55 +-2117305770,58026980659710872,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.509,1964-01-17,false,1067452366.33,13:57:6.23 +569387072,65396505255463760,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.509,1955-03-18,true,1150460857.23,6:35:58.12 +-841435275,43473945168574208,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.509,2013-03-02,false,154944414.734,19:54:56.47 +1886817267,29056534463834636,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.509,1964-08-01,true,1194273356.48,17:33:4.31 +-250764807,84125591002353216,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.509,1978-10-08,true,1206758505.97,7:46:41.17 +562858405,48400971918523864,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.509,1997-11-26,false,939347893.688,3:45:8.42 +-175992192,34088699452231332,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.509,1976-08-07,true,983380114.925,8:49:51.2 +-1003442188,37560078069696424,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.510,1986-11-10,false,851614441.181,1:31:22.2 +1111470997,39461463140836944,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.510,1985-02-23,true,528004424.329,15:25:45.26 +82372011,19700401584280404,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.510,1956-09-26,true,1162557570.49,21:28:21.20 +1564745709,90573982475348896,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.548,1960-10-21,false,746509920.537,2:49:2.17 +-1987421718,89790870276926368,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.548,1991-02-06,false,205818930.906,6:52:20.28 +-1471891938,69635751012635048,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.549,1950-09-23,false,861542213.654,8:52:50.37 +116865543,57447547891410568,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.549,2002-11-18,false,1301608789.07,4:46:43.16 +1362564199,16747096269591654,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.549,1969-08-15,true,100887680.754,20:48:23.24 +-1945229539,86601936537613392,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.549,2000-05-19,true,532996388.54,15:57:9.58 +1977008506,15752270795516262,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.550,1960-09-05,false,341601024.996,4:25:29.12 +-1747184805,46018822226937184,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.550,1985-06-21,false,853350869.668,7:55:30.7 +180568905,6058971799229829,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.550,1976-10-27,false,944441784.632,21:57:16.49 +1838882884,54851264097887336,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.551,2007-05-14,false,680279972.847,4:30:41.48 +-28236889,86485887626225728,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.596,1970-02-27,true,435573570.529,3:41:21.40 +-459077361,71068996419669832,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.646,1999-09-16,true,692324537.36,8:5:56.1 +738072141,21212080263113164,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.647,1988-02-09,true,82203304.7611,3:15:48.37 +-222947946,21301339785100840,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.647,2003-07-15,false,579750209.842,6:50:57.30 +1282178868,10534524664370718,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.648,1962-10-13,false,75538127.796,18:44:25.3 +-1184579721,89163225819174512,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.648,2008-11-11,true,1258838156.89,19:3:7.2 +1390821843,10036847698049966,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.649,2013-06-01,true,1113886216.02,14:21:56.44 +-2041248119,33921029747569120,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.649,1975-05-20,false,935358825.332,15:1:25.39 +823486875,30071193143980268,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.649,2010-04-22,true,1381507093.03,21:33:4.36 +-1141953170,61169496310703808,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.649,1976-10-23,true,1097581235.64,1:53:4.54 +-1754038661,19152697585176208,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.650,1994-03-03,true,416506136.669,4:15:36.34 +-778424877,67369595355817152,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.650,2001-09-25,false,1213318070.43,20:33:18.8 +-1578469231,34511800366797660,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.650,1998-08-18,false,1292664030.83,22:46:52.35 +-763261080,88565966639301344,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.650,2010-10-27,false,200277280.163,15:7:8.29 +681907126,37341493531391880,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.650,1950-06-07,true,105835672.163,16:45:23.6 +-1799524565,34728659981391596,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.651,2012-10-27,true,1256860791.89,21:45:21.2 +72196391,41094913815725832,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.651,1996-02-07,false,145632348.292,3:30:55.10 +-1154590183,35275782701537252,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.651,2012-02-21,true,888111078.162,16:57:26.13 +-939360359,60478845240087576,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.651,1980-07-08,true,354005853.048,1:8:27.23 +-677250535,83437274702632608,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.651,1951-09-10,true,355817377.862,9:39:32.43 +2088609004,33730885754036552,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.651,1963-08-25,false,1044488658.28,11:7:51.4 +-2053121368,85149505275360976,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.652,1993-07-19,true,1063496580.11,20:40:41.18 +37759319,64583164470630000,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.652,1958-11-18,false,876291116.502,8:53:58.49 +199474203,90194045076261536,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.652,1989-04-24,false,270308181.692,5:30:40.34 +310836480,28809429080141324,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.652,2000-02-24,false,11280515.5142,4:20:22.28 +-1582876525,73700930995975072,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.652,1956-03-08,false,388715872.233,4:14:50.51 +1949739487,84603384103416992,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.652,2002-02-01,true,1008646093.99,10:19:1.26 +-341072140,27768929603303744,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.653,2013-02-02,true,46545637.1823,11:43:9.44 +-660022550,55974054747936496,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.653,1952-02-04,true,620155855.702,14:11:42.50 +-1096081150,51494195442579264,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.653,2001-11-08,true,1217390196.15,11:15:6.16 +-258190329,16228132131396454,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.653,1969-11-24,false,746718597.744,22:35:21.51 +-2047947694,91038109545400288,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.653,2008-04-11,true,480816271.557,17:37:42.56 +-1464033537,59931440375027256,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.653,1975-03-12,true,78624742.2023,1:58:13.38 +1927219659,21466535712874580,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.653,1957-03-13,false,605123297.389,17:44:50.25 +-198504549,15001601113573694,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.654,1998-08-21,false,1121822702.27,15:17:32.48 +-707558435,37254479519678704,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.654,2012-10-07,true,1195041784.76,19:55:35.37 +-436141670,76704132678126160,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.654,1994-01-09,true,447747261.045,9:35:29.1 +1945171350,14711861909019320,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.654,1967-06-06,true,122551906.389,1:21:37.1 +578314815,82703900959707168,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.654,1957-11-15,true,1123382727.99,22:54:2.15 +-317151081,37719697313440792,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.654,1976-01-25,false,912977349.069,21:47:17.34 +822796668,92127973557919776,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.654,1964-04-25,false,570418323.417,22:40:58.51 +1136332912,179001805582254,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.655,1963-11-03,true,36978165.5073,11:57:34.18 +1528314208,76006789007498288,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.655,1989-04-23,false,891296562.628,5:47:29.51 +-2023721357,14613201523252470,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.655,2000-10-21,true,395695086.376,8:55:4.29 +1536490112,35595674625760952,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.655,1976-09-07,true,507640308.628,18:38:34.6 +483197815,4719342267731057,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.655,2001-11-07,true,688680053.643,13:46:3.43 +1190916023,3379808108966912,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.655,1988-07-11,true,602766093.812,1:5:2.46 +-1178305406,87245119144020528,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.655,1993-01-17,true,242835828.978,9:21:6.5 +-200967379,54698053295601768,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.656,1995-04-09,false,960484441.058,13:9:21.20 +1374368866,66633380167719856,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.656,2000-09-10,true,868095672.566,12:21:27.12 +-1611748115,90848041242603712,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.656,1952-08-23,true,101197044.364,18:8:43.34 +1357619730,18352491017629420,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.656,1970-08-01,true,441982818.374,16:22:42.17 +-1267200943,70166654224581856,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.656,1992-05-01,false,469695019.91,2:55:19.37 +-1244720122,41131416144515040,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.656,1972-01-21,false,569184593.476,4:38:58.47 +-1877926163,38883428458621720,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.656,1984-11-22,false,1430666030.1,20:4:44.19 +78222101,64881842061563376,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.657,2009-10-18,true,9434346.15815,22:34:30.49 +-223663586,16707287753798524,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.657,1953-03-07,true,721618768.748,22:33:22.8 +1792570558,70784075536173440,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.657,2008-08-23,false,32720439.5863,8:14:23.17 +1373108854,37998571723953400,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.657,2002-03-07,true,1200013215.33,17:37:33.58 +-502692263,38056763368697952,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.657,1976-01-19,false,1194147881.72,5:25:6.58 +2118823158,72035009063624704,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.657,1988-05-25,true,919878333.587,7:37:48.4 +-1804448365,85479412448361792,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.657,1982-01-08,true,720248849.639,5:47:8.1 +-989417134,51175482098821312,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.657,1968-07-22,false,316622735.918,3:55:29.40 +-1882897878,66491586806787024,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.658,1985-08-22,false,470683026.947,4:23:3.57 +347598838,89383397326296864,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.658,2000-04-24,true,771164610.989,1:30:17.45 +1529646114,58163592006430360,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.658,2013-07-18,true,785506424.571,12:30:45.46 +904828952,72861490527972864,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.658,2011-01-09,false,877339039.232,13:36:51.39 +-888573538,88775283024107440,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.658,1963-10-05,true,1168519456.78,21:54:24.54 +-247096104,67860560480501104,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.658,1994-10-11,true,1169122005.8,11:20:4.21 +1330102174,660143968041195,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.658,2009-10-18,true,234727424.363,10:55:1.17 +-913397744,33508157762554500,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.659,2004-10-23,false,595880780.419,7:17:39.16 +2132296083,68546329228986576,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.659,1954-08-08,false,1037637291.67,8:30:29.41 +-597866913,50852026188518968,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.659,1986-04-02,true,902204304.105,14:48:33.19 +-884576400,73491107459308176,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.659,1972-04-12,true,1396426811.1,14:27:11.7 +-93097335,47740252834116520,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.659,1961-02-06,true,671447070.596,1:43:8.58 +1638005326,31439442087934864,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.659,1981-09-05,false,705720861.501,16:8:58.57 +2085358271,8111211040816978,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.659,1978-10-10,true,842392478.98,1:22:32.45 +1489464807,71595805337279712,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.659,1967-02-08,true,533646259.347,5:26:40.12 +-1071767022,5930092965871831,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.660,1955-09-14,false,618475550.249,8:51:51.15 +-521474129,78173792153482208,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.660,1978-06-25,false,511318194.752,8:39:52.42 +514290075,6688476115650929,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.660,2006-02-17,true,1000875086.21,14:47:53.50 +226781184,64091462142660264,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.660,2001-02-08,true,1390949509.76,12:41:54.54 +1423294026,73410915091512480,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.660,1973-05-09,false,1389395041.98,16:20:36.16 +562435214,55572315583428848,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.660,1989-03-10,true,629465543.382,5:39:17.41 +584499571,35874376594107136,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.660,1993-04-18,true,179686472.688,2:11:8.19 +-2032578845,87684393808881088,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.661,1983-06-25,false,668607978.393,11:3:6.55 +-1846278145,38393356901338288,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.661,2010-07-15,true,713113067.661,21:28:29.23 +-995174316,22955630152120228,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.661,1954-04-19,true,668816154.33,16:40:37.30 +3241773,76713077818542288,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.661,2009-04-27,false,667496772.573,19:52:6.13 +-820639092,6185739025170729,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.661,1978-04-07,true,1220086144.57,5:23:58.36 +-52342328,10438111450777426,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.661,1957-04-21,false,19201675.2008,11:38:11.37 +622273415,63765441753338544,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.661,1998-01-23,false,903038025.277,9:19:21.36 +-2037371124,21973107902850652,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.661,1966-06-20,true,1156420374.47,20:19:2.26 +-369538290,90595761042060816,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.662,2007-09-16,true,819583730.516,2:2:30.43 +750350150,29950808075288680,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.662,2010-08-09,false,498710430.565,22:50:55.31 +-189772069,23691755202279300,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.662,2005-03-11,false,808066805.192,19:11:39.15 +-844583302,19423486740176324,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.662,1958-06-11,false,297396156.623,22:32:30.32 +1140928204,45551942275389552,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.662,2003-01-24,true,259787530.661,18:38:3.50 +-723262853,82508165315923600,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.662,1957-11-06,false,830121369.055,10:51:57.39 +1063084632,18683732617704952,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.662,1950-11-20,true,486025078.155,9:27:22.57 +-1515797359,22980692739520276,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.663,2009-01-12,false,1429925116.48,1:31:43.47 +1998205883,49443625775556504,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.663,1972-10-02,false,688578881.23,21:49:24.1 +2084891150,50164544407034008,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.663,1993-02-07,false,436936241.11,4:39:11.21 +1042460846,49234427826781752,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.663,1957-10-03,true,306231482.324,1:49:13.55 +1344561078,33620547416345908,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.664,1961-09-11,true,243733578.694,16:41:11.39 +-2108874766,88870902004115872,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.664,1976-07-23,false,1306707084.51,5:56:24.13 +-214462111,51375239096915712,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.664,1975-08-01,false,1426219105.7,17:26:41.22 +-1432621210,27458099693574256,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.664,1968-05-06,true,178957392.412,13:56:14.10 +-122108492,42499083955690712,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.664,1976-11-26,true,1283881017.45,6:54:10.17 +-227468387,3147782077047286,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.664,1986-02-15,true,954120624.143,6:43:48.53 +1958845636,9707211958146540,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.665,1981-05-05,false,71239959.3235,11:3:11.49 +321437519,69044707632950816,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.665,1986-02-02,true,290533308.233,17:42:43.52 +-1941534154,60460090380477248,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.665,1978-01-06,true,430700058.072,20:48:7.47 +1258554049,7488547406404301,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.665,1955-09-10,true,975021386.737,10:40:23.40 +1654122818,78728182859774704,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.665,1964-03-14,false,346611908.105,4:56:50.58 +-2001415665,30109448193291388,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.665,2002-11-27,false,903668241.598,1:26:51.17 +692931971,64545986074053520,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.665,1979-04-11,false,540575441.871,19:46:21.42 +-1417053768,4047031589679206,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.665,1980-09-13,false,2710695.48749,20:12:35.30 +329859800,83512949459583616,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.666,2000-03-25,true,688384050.813,13:4:23.49 +-863510620,21548462533458400,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.666,2002-09-15,false,220561659.58,21:47:4.17 +-424920846,20174844562983404,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.666,1960-02-03,false,874827285.385,6:22:38.32 +150493955,63812342297245256,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.666,1991-06-03,true,960585647.529,16:43:29.26 +-1613418574,1010626234327439,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.666,1966-09-24,true,1028954356.71,10:53:13.49 +190181930,9322127878236734,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.666,1988-09-14,true,561042839.943,9:28:27.21 +-816064327,91703669011373728,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.666,1983-08-15,false,1233546802.49,21:16:18.14 +-84711216,71797908687022392,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.666,1967-09-03,true,553060951.874,19:9:6.8 +1240524247,68891973225703176,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.667,1970-07-07,false,1319881931.38,20:32:13.14 +-1430463599,3200681130277048,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.667,1956-10-22,false,491417790.607,7:56:18.37 +484944458,48496898469347768,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.667,1985-01-23,true,585054005.872,21:13:37.24 +1633562001,23462969936748064,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.667,1995-02-15,false,417834981.142,1:17:17.31 +-113287175,18143105302707980,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.667,1992-04-12,false,1009394030.26,15:6:30.43 +-1103371157,70531173061283656,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.667,1965-09-26,true,1137329926.84,15:56:49.29 +1324472745,53304577433172368,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.667,1980-08-07,true,458251692.723,11:56:11.22 +-1111447329,85257573946864912,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.668,2007-01-02,true,1376293293.44,18:6:53.23 +-1382656347,15504409504424592,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.668,1954-01-07,false,859446696.651,8:24:9.54 +-1816332801,50942869757879104,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.668,2008-07-26,false,1008098909.42,16:7:40.44 +1465588656,9413437791548406,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.668,1996-02-08,false,1086152056.23,14:29:39.24 +1005452881,64664398830452400,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.668,1969-08-10,true,925373598.139,9:54:28.50 +531980419,42755127018210040,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.668,1984-07-01,false,905320034.049,1:25:45.7 +-633514662,66144302162302912,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.668,1965-07-24,true,398043318.141,20:50:7.38 +-560594081,72349939500354320,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.668,1981-09-18,false,490628711.33,17:7:15.3 +-796739150,103995001377720,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.669,1975-04-19,true,1300954203.32,6:55:15.3 +-814595869,79859108581875776,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.669,1992-11-04,true,857112504.791,9:55:31.39 +1528852874,57274146399490552,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.669,1956-05-06,true,303516143.594,9:25:46.50 +400653873,13968579855241554,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.669,1993-01-20,false,1046048704.28,18:51:1.22 +2021808869,18454131504178504,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.669,1998-02-13,false,1152068341.66,22:31:29.43 +-252341998,66434647901486704,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.669,1984-03-25,true,1092075029.92,11:1:5.41 +429141002,36004133362514424,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.669,1970-02-23,false,1235087686.11,10:14:56.37 +851738859,50972714427335968,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.670,1996-08-27,true,716943837.916,12:23:39.53 +-1276510324,54471137944187280,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.670,1965-03-17,false,75193845.2009,4:51:58.27 +775113820,27031758397928716,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.670,2013-11-13,true,789326635.319,2:42:43.36 +682195804,17881535930440786,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.670,1963-06-13,false,581144633.835,19:8:23.11 +-1444189321,14009474904181780,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.670,1952-04-27,true,100157944.229,17:46:45.34 +550194570,88830381634585968,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.670,1960-08-02,false,1136055987.88,6:20:23.11 +1946768409,46339996558403104,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.670,1955-05-12,true,334073435.236,3:44:42.3 +-2068024111,87428830963120032,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.671,1984-06-24,true,138832855.816,2:47:16.6 +2044094261,84756942763112608,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.671,1958-05-17,true,847869561.443,3:14:50.28 +-1844351010,69470453046336368,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.671,1982-08-04,true,79772177.168,11:6:7.33 +-2142354151,24063478858193428,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.671,1978-03-02,false,1348865725.72,19:27:52.24 +625500433,48633905488563648,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.671,1978-05-19,true,738374131.805,15:40:31.4 +1092041685,73560798905181264,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.671,2003-08-19,false,961196867.92,4:28:51.42 +-937840457,59769571983516368,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.671,1976-11-18,false,1001439971.38,17:15:58.44 +-596580334,28948150073240628,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.672,1974-10-15,false,813289336.943,17:7:58.41 +591725476,82560416492887344,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.672,1960-09-06,true,736602280.488,22:8:42.19 +-918948218,6338401823423314,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.672,1999-05-02,true,963774977.456,8:1:5.6 +-1423008790,74537618096107888,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.672,1951-09-01,true,60105749.2539,8:19:17.41 +-1405294567,21587325411814716,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.672,1994-01-12,false,633088727.909,18:39:35.16 +911739030,13464084246961398,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.672,2000-07-19,false,439355258.633,18:37:27.21 +1010550275,12282701534885418,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.672,1953-08-17,false,172437517.443,11:30:31.16 +-153496441,30241893523509116,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.673,1970-10-10,true,757782853.24,11:33:39.5 +-2058979866,27687881435235184,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.673,1969-05-27,false,406041193.121,19:45:20.57 +2023000529,85028758549006848,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.673,1971-01-21,true,718843481.285,13:50:42.18 +2099705140,62903497487410896,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.673,2011-07-03,true,39998931.9916,13:34:44.53 +1238726857,28648137599414660,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.673,2007-10-03,true,841715677.146,2:8:1.20 +1321422270,10596189336323420,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.673,1992-01-06,true,1314312950.9,8:39:35.51 +-1680246265,47659278233471096,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.673,2006-02-03,true,726242544.973,22:16:3.33 +1996714111,14750559935304294,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.673,1954-02-08,true,64865929.6364,11:17:35.22 +1566118525,24938039980075756,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.674,1982-09-24,true,1386326464.91,18:10:23.41 +1822463997,49562295776853568,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.674,1975-06-16,false,1349822734.45,5:29:56.19 +2032344559,27436113405345760,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.674,1994-03-24,true,784345210.562,10:13:6.48 +-731498429,24432457164489492,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.674,1998-08-16,false,82567783.3474,14:53:21.51 +1687439336,2087983368094412,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.674,2006-03-03,true,308200404.435,9:27:25.25 +1145077996,90119853539908624,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.674,1976-03-18,false,46754203.9726,2:48:57.42 +-731280027,26731890188210912,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.674,2007-11-19,false,70515781.7632,11:46:55.28 +54795488,50033586185349120,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.675,1958-02-27,false,272676475.781,19:45:46.43 +1163893597,8099202046028329,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.675,1978-11-23,true,627028240.255,1:44:48.42 +-1161476436,74538674541284240,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.675,1973-06-12,false,1051159224.26,8:13:4.49 +-1634386302,74045280218118416,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.675,2010-03-02,false,469780384.046,7:2:43.41 +-590414036,57240704666183024,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.675,2002-02-11,false,1178779464.25,9:54:4.24 +807935631,51064604411747656,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.675,1989-04-11,false,1385236725.54,14:32:56.40 +-771320910,40287389651231632,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.675,1988-03-01,false,848490342.901,17:45:28.49 +-216851088,8269966461544049,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.676,1973-11-17,false,989892654.827,21:27:28.33 +-260718033,73313620678994912,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.676,1998-06-02,true,885322479.878,18:51:50.23 +715745320,75226374518253600,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.676,1958-06-27,false,30374267.4467,12:4:58.15 +-1733945632,50081865120293040,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.676,1975-03-12,false,1404674573.46,1:51:27.11 +-1422220794,91215427489546000,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.676,1996-10-08,false,345219575.549,1:1:3.13 +-1801784965,9995816793359566,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.676,1964-04-20,true,626693562.966,11:23:25.19 +909306336,27843375457629052,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.676,2009-05-18,true,511932824.415,18:51:15.7 +2111667859,84650603844880432,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.677,1961-02-23,false,918891187.5,3:57:49.12 +836617561,45281390720073216,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.677,1975-06-15,false,57154930.7166,16:31:24.19 +1365564077,47046601404478728,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.677,1981-09-06,false,803755160.706,12:43:32.50 +2068588629,62849005410823304,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.677,1976-10-15,false,1097242425.52,11:2:9.39 +-2004027165,33333732153854096,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.677,1959-09-26,false,919600845.721,18:2:40.42 +193114643,28763539834006436,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.677,1971-03-12,false,1178105491.47,20:54:15.52 +776176489,14098259397709272,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.677,1965-04-04,true,1374278447.63,6:25:42.58 +-142108256,91286789510459536,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.677,1974-04-08,false,506769690.76,7:50:10.38 +-498002089,38114897749378040,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.678,2013-08-26,true,191632138.447,8:15:20.43 +713204477,29833198580931340,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.678,1987-09-24,false,632413113.176,5:6:5.46 +-157751394,69961376752231040,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.678,2009-11-12,true,232811770.826,21:39:42.12 +1932622673,72110871155333616,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.678,1977-04-08,false,539055052.372,7:15:1.17 +942062568,87321922899737200,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.678,1951-03-17,false,233245117.402,4:9:51.5 +1440562817,50237590122882808,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.678,1995-02-26,true,530728129.356,12:7:52.17 +-324802539,69554538108453744,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.678,1957-03-16,true,403767544.823,20:4:13.31 +-912399414,65392218380182952,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.679,1950-03-22,false,350325741.698,19:58:49.46 +-1819587600,68656658561987424,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.679,2013-06-19,true,884408458.261,11:48:11.31 +-957717498,53508365800150080,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.679,2002-10-22,true,1318917547.15,21:26:18.36 +-1660893244,8210111901097063,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.679,1994-09-04,true,1022833923.04,11:25:27.11 +-908616179,65793614095559064,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.679,1959-02-20,true,398845321.179,3:33:23.49 +1474919701,18607608983806792,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.679,2006-11-16,false,1374134999.54,3:38:42.46 +-1011382349,68843038591756184,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.679,1980-08-06,false,178803110.308,6:24:20.50 +-2004814117,87256889175465712,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.680,2011-11-17,false,469773982.162,19:36:35.10 +1324040490,56042322903446840,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.680,1993-07-07,false,639708497.035,20:46:39.20 +905429386,65698942792967432,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.680,1970-07-06,false,22235831.8036,9:15:16.54 +1339794012,53521113781442168,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.680,1980-09-05,true,537490352.415,16:10:50.5 +-456309996,40642107591889960,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.681,1971-08-06,false,402031306.259,14:15:1.31 +619304601,36657452530222176,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.681,2012-08-20,false,1342392023.47,3:38:10.13 +-1183474647,23010668046510192,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.681,2005-05-10,true,268164473.911,14:49:14.33 +-1286021993,1459271545871636,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.681,1991-06-05,true,124107409.293,16:8:55.12 +56790784,84760667695536720,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.681,1988-09-23,true,363178401.882,4:27:7.48 +-1862484414,54352207945644880,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.681,1970-11-18,true,642211438.601,3:9:13.39 +1706962435,27375821294311300,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.681,1995-04-09,false,1095341404.19,19:30:21.35 +-1923336805,58081304314759024,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.682,1951-08-18,false,427051723.101,12:31:13.15 +-1998869968,73660396895522416,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.682,1956-03-17,false,63986712.1302,19:23:18.1 +1779625729,23138507496508508,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.682,1966-03-25,false,163581327.6,13:21:38.19 +-168250998,44189141932883184,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.682,2001-11-19,false,117603631.647,4:17:56.40 +1421038719,36773398956924000,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.682,1971-05-17,true,635058525.163,12:33:13.18 +-648112991,53260132615997576,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.682,1991-09-25,true,54134594.0802,21:9:29.43 +-2065809600,36080293362026864,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.682,1992-02-23,false,583867238.729,1:53:16.10 +1677522624,54510897761092592,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.682,2011-07-27,true,982497712.618,3:44:17.51 +-1617133649,14794905939032126,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.683,1987-01-25,false,632033815.731,5:46:41.12 +-1075454017,17886601331114968,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.683,2002-04-04,true,1055413524.28,4:44:58.48 +-25163604,53542362334581488,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.683,1992-09-13,true,906832463.567,18:26:37.48 +-2081530376,75613033805212064,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.683,2000-04-03,false,47017287.4101,16:23:43.8 +-99194103,68318500110950880,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.683,1976-01-26,false,259769883.867,1:43:37.29 +-944840094,46172135302617184,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.683,1960-05-21,true,441459309.312,7:46:32.47 +1421664840,86456808403468784,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.683,2001-11-24,false,452120896.953,1:37:48.46 +1971691903,59487443863913536,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.683,1984-11-24,true,61363399.1254,17:14:5.27 +1438873766,9664758214227836,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.684,1992-10-09,true,691312847.003,17:1:8.56 +-282241901,44557252561408592,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.684,1964-04-06,true,301133226.373,19:56:50.58 +-2001097798,38493569337620192,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.684,1959-06-03,true,33479280.6866,13:31:9.55 +1775061260,32875251890187920,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.684,1971-06-12,false,1294445811.95,17:33:27.44 +-1239431294,54831080310823240,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.684,1993-09-22,false,1030337198.91,12:52:23.11 +-1104503497,26538612132030412,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.684,1994-04-21,true,1265371319.81,5:52:4.22 +-1315755108,79158784591475312,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.684,1960-05-21,true,568680040.144,13:12:50.37 +-793695949,13455270166297866,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.685,1996-09-05,false,1030240390.23,6:11:48.14 +-1471723730,21439643891340708,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.685,1953-03-27,true,519908700.623,13:45:15.38 +1989582907,86613620958901488,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.685,1966-02-20,true,206380636.3,15:57:47.3 +-2031626039,68858022744481936,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.685,1960-02-03,true,483324203.35,15:29:43.16 +-1281501186,89240997365184448,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.685,1959-09-07,true,1170606723.39,17:58:3.13 +-1830559507,4205947391684362,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.685,1968-07-20,true,31015638.9413,19:43:46.3 +-559978570,82514249612000784,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.685,1983-10-18,true,1176256567.87,19:19:54.37 +-837185527,22199052417950832,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.685,1954-04-17,true,814175235.048,11:18:4.31 +1750535616,53866651709425976,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.686,1980-02-11,false,535682783.255,1:51:39.10 +-1275211773,70438856308464608,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.686,1985-04-22,true,233685822.929,20:13:14.8 +1950377706,88266257020385648,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.686,1994-10-17,true,1273613240.41,20:39:38.26 +-668330809,44703726541342520,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.686,1970-04-22,true,276528539.715,20:49:8.50 +1308137515,70281508314764496,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.686,1951-02-24,false,339521249.913,8:18:32.43 +1019768431,85296257860912304,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.686,1974-06-14,true,752282024.926,4:20:41.9 +-457735259,10750625186152674,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.686,2009-10-06,true,890505146.995,12:40:50.4 +924286529,69869442189515008,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.687,1977-02-17,false,141737610.067,4:16:48.55 +-1616188068,9465255340042854,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.687,2006-03-18,false,323208633.656,9:3:37.8 +874084482,51007944033678920,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.687,1976-08-09,true,509214721.683,17:18:14.6 +1219897044,49743813254796568,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.687,2002-11-16,true,1049771332.83,19:26:54.45 +665730921,26390156988732816,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.687,1990-04-15,true,935445023.524,11:56:57.57 +1117958717,27189481942615460,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.687,1994-07-17,true,1362438366.07,20:42:31.19 +1604845272,71867747535026984,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.687,1950-11-18,false,943024748.34,19:50:41.38 +2060009609,12612715087130286,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.687,1954-07-03,false,320250291.314,10:41:53.27 +-1406141326,29138307696198184,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.688,2012-10-09,false,623784361.201,9:51:33.33 +1619103597,83362509765299648,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.688,1960-11-19,false,1088209620.61,18:14:57.45 +1714407851,19181261102411212,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.688,1992-10-25,false,1203193594.48,19:54:31.11 +519414432,55944370253479784,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.688,1951-09-18,true,598753169.034,19:43:38.23 +198393894,35063554956316244,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.688,1980-05-18,true,585089886.35,3:14:13.21 +-1934421504,1302838576872704,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.688,1950-10-13,false,1029310558.7,8:32:33.6 +701829056,66511127681361944,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.688,1964-02-03,false,209742608.869,1:44:22.30 +594747717,78418943250099456,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.688,1952-01-23,false,1428426384.24,19:35:43.45 +1742606071,78091640749538640,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.688,1976-08-20,true,1389138550.29,11:24:45.52 +-1675380128,31222287325488772,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.689,1958-02-07,false,235734164.115,19:50:26.48 +2020136812,45522936191712584,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.689,1959-04-25,true,253262147.131,8:51:36.22 +-403392635,32713417895688848,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.689,2010-03-09,true,1005796736.78,15:9:46.18 +-866225462,7045776005412209,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.689,2004-01-24,false,1108124870.15,11:22:48.6 +-349822897,53606522372039496,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.689,1994-05-09,false,752180759.219,4:26:22.57 +-722599672,1069172283015946,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.689,1959-04-06,true,151182441.176,21:27:20.32 +-1121813237,64257857829794488,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.689,1958-10-05,false,983682365.001,15:37:26.7 +-539705883,64762931389585320,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.689,1974-11-01,false,656402710.695,22:39:8.40 +-2146463863,47879550300972176,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.690,1987-01-02,false,1417498484.14,14:50:10.23 +-1489710160,24191786943602608,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.690,1962-02-04,true,607958618.037,4:52:24.31 +-1063326626,43773156024692512,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.690,2004-02-11,true,363765833.523,5:32:35.25 +-884997440,69555692211213112,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.690,1951-02-18,true,907509335.791,4:56:15.15 +-1626120899,68484543659306760,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.690,2004-03-20,false,267652004.093,3:38:58.17 +-957268668,61307636783981960,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.690,2007-01-12,true,201706339.305,2:33:16.46 +2092173270,67800467756766672,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.690,1965-06-21,false,476409105.349,2:36:48.36 +-1576466352,4202645621380813,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.691,1979-01-14,true,416620790.062,11:45:1.12 +-297407240,19231515079152324,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.691,1987-08-08,true,362779470.745,2:42:46.36 +1819430105,42187060121428296,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.691,1992-11-22,true,1416231259.05,18:19:52.57 +962195715,73976391526608464,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.691,1984-10-05,false,275584808.447,18:40:12.54 +1704892045,48621620872842208,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.691,2011-01-07,false,335613673.807,13:12:55.20 +1232145655,91994373251570656,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.691,1950-11-23,true,390511616.148,4:30:31.53 +-670959987,61987434343803496,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.691,2008-09-04,false,333039849.804,12:56:31.47 +423804309,22293040915467448,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.691,1981-11-07,true,384037103.389,7:41:6.20 +1240839294,78174246333081056,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.692,2004-02-02,false,1118907297.79,5:30:26.24 +422350181,22970516352757648,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.692,1971-11-16,true,16338774.8848,11:7:44.6 +-1293705130,67000315931076960,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.692,2009-10-22,true,161613159.288,2:8:50.45 +-1448196545,37844092533492336,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.692,2002-11-17,false,445028731.757,17:26:46.8 +1661642109,51469585490877624,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.692,1982-02-13,true,940921589.349,16:20:30.49 +-1709349374,9665293061781658,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.692,1968-04-19,false,201303636.661,20:7:16.51 +-723291507,9087785317831946,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.693,2005-09-10,false,546995468.534,3:3:58.1 +452395404,34835055891972844,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.693,1994-01-02,false,1048942606.67,11:22:58.50 +-858134875,35850709731127716,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.693,2007-04-06,false,542634039.49,10:19:18.54 +-284834437,44035640594584256,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.693,1968-11-22,false,674793659.822,13:6:25.34 +100938026,5172680159246439,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.693,1952-07-24,true,1201411682.63,16:41:26.34 +1206966493,31926884982155328,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.693,1995-03-27,true,1260209188.78,12:23:35.28 +2060684009,14505414712164588,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.693,1988-05-19,true,816632598.338,11:2:56.15 +-1901647300,19221884480819344,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.694,2006-01-18,false,27577799.1712,2:12:50.43 +-2016365917,68196478767265008,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.694,1998-10-16,false,410830438.189,17:34:27.5 +1360349545,51092938458613000,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.694,1983-07-04,true,632785213.638,11:47:1.47 +737208416,58206808671038136,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.694,1960-09-24,true,1256590716.06,10:55:20.56 +-632119085,66910784873324728,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.694,1995-05-23,true,995900182.034,6:30:30.38 +-634003195,51311007577579528,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.694,2006-08-25,false,974259087.355,16:48:33.32 +-1845290674,58093182286940112,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.694,1965-04-06,false,786688343.098,10:58:18.22 +-1027316318,47540996158440688,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.694,1994-09-24,false,159998953.121,3:58:22.52 +2120275887,52473199644437544,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.695,1961-07-06,false,1101932565.51,4:34:20.14 +-697977919,82346992333822336,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.695,1953-03-10,false,394307761.274,7:30:17.42 +929021992,55910127274904960,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.695,1956-04-14,true,219474809.908,19:51:1.46 +-27888034,48514722296611320,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.695,2000-07-06,false,1022248113.6,4:24:17.6 +-1772887622,42206772126626000,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.695,1963-11-09,false,1140248170.28,21:37:17.21 +856179570,16949335845899408,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.695,1990-04-05,true,629114311.508,12:38:34.28 +275060064,39904263878566656,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.695,2006-07-20,true,581263421.548,20:22:10.21 +-390791484,48014628322350080,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.696,2013-04-15,false,1089239376.28,2:49:19.11 +-1742952468,63394755904954624,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.696,1986-02-10,false,8990836.20821,13:36:25.43 +2095247141,38557099097726568,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.696,1976-03-22,true,286085714.647,20:41:19.32 +-1670992572,50724195734429504,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.696,1973-05-21,false,385845467.737,4:54:32.18 +-2132306204,64072512035911592,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.696,1955-06-24,false,465055108.213,16:17:21.53 +314651780,60322703863324312,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.697,1955-10-07,true,1270182123.81,7:27:48.29 +600139894,24218211033298748,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.697,2012-04-01,false,816183870.71,8:54:16.21 +738532464,31022999259391316,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.697,1960-09-19,true,402551595.163,10:32:40.28 +328982908,39700954473269272,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.697,1995-07-16,false,1296239895.87,11:52:51.15 +1492554222,52363881368351256,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.698,2008-02-19,true,903266322.96,9:12:40.30 +1744159723,20646946957407764,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.698,1960-01-25,false,1045015192.19,19:45:20.56 +-1752749383,32297434151281060,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.698,1982-08-21,true,1337324515.71,10:48:7.44 +-854983177,3807652500574751,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.698,1962-01-14,false,899198143.134,12:26:58.34 +1910983215,45295050666072792,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.698,1958-05-04,false,329615400.467,6:35:10.49 +1698952102,68738925583873336,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.698,1960-05-05,true,848390796.19,22:46:47.20 +1448105325,77937317939966192,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.698,1995-03-07,false,805144800.385,12:49:46.11 +-134092621,90914289338467936,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.699,1983-02-09,true,718137903.596,4:42:54.6 +240144834,31915362448173528,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.699,1977-11-10,false,1103850307.52,19:7:18.45 +1023961492,20620268912031652,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.699,1973-01-11,true,486251223.078,21:20:24.43 +1046079238,24504984795700992,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.699,1994-11-06,true,452873438.819,11:46:12.11 +649125337,14376087762723594,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.699,2011-04-05,true,1282681763.42,11:13:48.24 +1040801033,18673724310620488,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.699,1968-05-14,true,143428250.835,4:26:49.27 +-1850040112,15592076294455204,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.699,1994-04-17,false,1053441983.48,13:39:17.7 +1468753004,22443444562935544,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.700,1958-02-08,true,964867537.368,20:56:24.5 +299447013,46956736107880360,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.700,1959-02-04,false,1202476360.94,8:58:6.9 +-1064284523,53314298829932896,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.700,1971-11-15,true,463784200.234,9:1:50.38 +741579638,22303743097409844,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.700,1993-01-03,true,848879472.822,1:11:15.57 +-1055617680,80299087533602288,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.700,1961-09-24,true,316510806.233,6:19:23.24 +-2143072617,38571400324794624,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.700,2006-03-03,true,48947627.475,10:1:42.2 +-2006548252,56013329605291320,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.700,1985-01-17,false,24355953.7663,14:24:13.3 +-1013709121,8422731582112717,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.700,1995-04-14,true,1176082050.57,15:47:31.48 +1008673014,86098073824314240,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.701,1964-11-18,false,1154700313.77,21:33:29.37 +-446123786,9731853433337140,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.701,2000-07-22,true,182176775.8,15:15:15.47 +237133225,74720023116678960,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.701,1981-09-16,true,719998007.535,20:49:29.4 +834557189,62671814449258336,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.701,1966-03-02,true,1071122053.09,15:56:24.37 +126659301,47857212510692456,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.701,2009-09-14,true,541011943.908,14:48:26.3 +-762614244,36832739175424360,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.701,1976-02-08,false,242165718.33,13:11:31.55 +126115488,5469850330680474,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.701,1989-06-15,true,539706459.935,5:56:35.44 +1612499235,88340508890582464,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.701,1966-10-02,false,23386943.5183,7:14:49.57 +1312761820,57101575907629280,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.702,1952-09-27,false,591761668.026,15:36:26.53 +919053816,7198009690337741,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.702,1998-09-06,false,1062161147.44,8:7:6.44 +-1440665582,74141138229448128,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.702,2012-10-24,false,466739466.668,14:42:42.49 +451724346,11775048379714202,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.702,1981-01-21,false,845138163.001,12:8:54.8 +373180623,3891889971606589,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.702,1981-03-15,true,780894320.083,3:28:57.8 +-592421580,58275316110985640,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.702,1974-11-04,false,659289189.084,2:27:28.29 +-1810229822,3291229967525488,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.702,1998-01-04,true,1153714021.61,7:52:1.31 +801629533,91051315839524480,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.702,1996-10-27,true,1258166831.04,14:39:8.36 +1591080884,45349277171774424,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.703,1960-09-20,false,704166258.154,4:56:55.19 +1069517552,68635945597798576,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.703,2007-07-07,true,1286577738.2,8:1:11.44 +446939649,90504540200114000,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.703,1977-09-14,true,9876143.99858,21:4:43.15 +-1087591114,14561573330249596,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.703,1981-09-05,true,622029242.865,17:31:45.27 +471226264,75073957697134464,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.703,1996-02-04,false,516825387.239,4:24:44.18 +651585745,36078610973387768,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.703,2009-08-21,false,76461321.3812,16:15:3.6 +1357317702,51958517056986616,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.703,1955-07-23,false,1366502954.94,2:13:17.30 +545227845,61708998612882392,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.704,1960-10-18,false,37293823.9902,9:14:42.50 +2131635815,84982167566546912,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.704,1972-01-05,false,1403763877.25,14:4:38.48 +-2060074755,77850649022879136,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.704,1963-10-27,false,134142048.345,6:30:25.57 +480916483,85117362452613616,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.704,2009-11-02,false,995745697.16,3:23:53.52 +-2098303331,68578041195889720,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.704,1999-06-25,true,1062294491.16,21:7:38.45 +-1197399431,8997352969811589,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.704,1988-07-09,true,634873993.382,22:39:14.55 +-1275954446,7495604317208085,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.704,1983-04-27,false,222643177.202,22:14:41.31 +-2022197993,12656415594706832,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.704,1959-06-19,false,312244504.413,14:3:17.5 +-608754419,33458344602692404,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.705,2007-06-15,true,1344522188.47,6:12:28.40 +1679641847,74619046854211264,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.705,2002-09-04,true,1133862190.56,1:24:53.50 +-12861873,6984319344503081,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.705,1968-08-08,false,1319980624.28,18:16:22.27 +1077772969,59021570124041176,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.705,2005-11-22,false,828589367.896,16:8:23.57 +1972996045,47951255201767920,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.705,1998-06-26,true,693551458.977,17:18:17.29 +542614760,76896284317626864,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.705,1971-01-03,false,1432767593.85,21:4:38.38 +1320080929,22498750843302256,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.705,1977-09-19,false,842705338.4,11:53:42.28 +1951693586,70280774770026464,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.706,2012-05-01,true,1299605508.49,1:29:40.32 +-1366103004,68871184401174128,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.706,1977-11-21,true,886058282.363,2:31:17.10 +683842950,17062998528088608,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.706,1969-01-25,false,1276222671.97,8:14:36.49 +-447232492,24363967275749624,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.706,1950-06-03,false,210450234.1,2:18:35.53 +113173354,43241151911612240,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.706,1966-11-19,false,1381943521.66,14:32:25.2 +-661183,58972487212822424,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.706,1962-01-04,true,836726865.299,21:40:14.45 +196380790,91628553306190496,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.706,1973-11-15,true,416122682.711,11:37:39.23 +1583355167,11710873696716534,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.706,1966-04-15,true,895603722.019,17:31:55.23 +-1957924767,47204622846082496,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.707,1964-09-18,true,929837733.311,18:16:40.37 +1472882235,67860233874331432,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.707,1960-03-26,false,372666501.893,14:9:3.20 +1677433136,90342155297158768,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.707,2004-06-07,false,513539895.706,20:3:41.40 +-1205194777,56857898541311448,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.707,1959-08-01,false,741260920.534,13:30:54.38 +-93176731,52661049612494616,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.707,1968-03-21,true,780819340.208,15:23:32.23 +-975692979,83920513933381856,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.707,1976-02-13,false,363888773.031,16:34:57.22 +-852721726,31227100238278596,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.707,2013-06-20,false,1123132610.41,12:44:3.9 +-49905158,12674108582774210,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.708,1965-03-08,true,1382391736.99,3:29:49.44 +-1590597311,61224205253533360,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.708,1965-09-23,true,405191582.259,11:50:21.28 +-127531374,90740437116183376,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.708,1963-04-02,false,357745895.334,16:27:5.25 +1028734111,65510569407378880,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.708,2007-05-04,true,408085376.399,7:15:45.44 +165008085,7996141677694864,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.708,1988-07-07,false,283476370.586,9:22:1.43 +25489840,22173499705922336,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.708,1993-03-14,true,1021968528.45,16:27:2.11 +-69192771,36932220381787496,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.708,1999-01-01,false,1344593303.46,18:51:20.51 +-1776188874,54956138367096704,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.709,1985-06-07,true,1282663867.32,5:2:46.31 +-1730048493,35449978015282288,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.709,1954-11-24,true,562529907.282,7:11:43.13 +294733700,26961151381998888,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.709,2009-02-21,false,642450746.218,15:25:37.35 +-717357954,9537331199189822,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.709,2009-03-20,true,1157697518.98,1:50:30.16 +1744520007,58663487394778072,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.709,1994-05-27,false,83128993.1281,14:13:37.12 +-112543677,73033992380667792,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.709,1965-06-03,true,751911247.748,12:42:7.17 +-1013520266,13731109230128784,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.709,1963-09-03,false,602328041.89,1:18:43.20 +188148195,1769464213784330,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.709,1973-03-02,true,106780169.708,13:55:3.26 +1437256025,84155712386249376,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.710,1954-06-04,true,613533774.626,20:53:42.36 +102456150,56011395690647264,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.710,2001-02-21,true,884970152.072,22:24:56.42 +-1251934087,12682173467866430,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.710,1952-04-24,true,11274137.7941,2:8:15.20 +-543473154,80598729133496944,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.710,1975-07-02,true,668067676.653,11:7:39.50 +1345172860,34000760561480664,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.710,1957-01-25,false,1256292551.64,5:16:53.52 +-29060408,1726655468892590,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.710,2003-04-07,false,340033049.95,21:3:4.37 +-6278621,34883690389504636,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.710,1999-10-25,true,1344654922.65,18:29:40.9 +722779914,1280991192192010,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.711,1999-02-07,false,580475296.708,8:50:46.14 +-440529495,10061724165172234,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.711,2004-03-08,true,790109270.893,14:50:13.12 +1708821380,28118087342112524,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.711,2009-08-09,false,29544314.359,20:16:18.39 +-1171760429,54350566151900032,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.711,2002-09-03,true,457173838.32,7:5:8.35 +-269822087,17764650806224364,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.711,1989-07-21,true,1232097065.98,8:29:49.42 +-1710086628,44475023843279752,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.711,2013-04-23,true,372161440.387,8:6:39.3 +1213173869,72195242112327520,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.711,1996-06-07,false,1355978569.28,2:28:49.7 +-1538329954,64339592420977464,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.712,2006-03-10,false,1401289677.85,8:44:34.29 +389639589,29545403449230368,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.712,2009-03-03,true,411633569.905,20:27:6.43 +490963036,71301223707906328,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.712,1985-05-02,true,917253911.793,4:6:2.1 +1018209945,54113397001012480,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.712,2003-05-17,true,144763602.232,10:4:31.41 +-468595626,39970394324308704,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.712,1950-03-16,true,583747840.408,8:15:21.26 +-1483601044,3499345067257456,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.712,1977-06-05,true,8200078.47424,2:11:16.44 +1392598867,87934657458434448,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.712,1954-07-20,true,14495263.5759,9:24:34.36 +472812462,3323014408738898,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.713,1993-02-24,true,306938030.18,18:39:51.16 +-2018913323,39519808641281912,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.713,1971-11-20,false,950458540.436,1:58:54.6 +596163913,24744207776081696,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.713,1998-07-11,false,223669269.899,11:7:45.35 +751091073,62382446657979176,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.713,1967-04-15,true,1156938543.94,21:4:15.54 +-351506996,89414189266929936,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.713,1950-11-10,true,778391628.874,3:45:49.39 +-1328498908,16602956670142576,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.714,1970-08-04,false,34039993.0376,17:31:15.8 +-81492475,66390464380848376,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.714,1985-03-11,false,361433418.885,4:25:48.36 +1677728730,2398026029874084,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.714,1950-06-27,true,1221864169.0,2:47:55.24 +-149823794,88964038047138640,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.714,1950-06-13,true,720459907.527,17:50:8.7 +-126095429,35128394672041176,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.714,1958-07-09,false,629381683.095,16:58:30.49 +1593953325,33402810524632996,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.714,1977-04-09,true,257974754.658,17:1:51.15 +1178823750,24074922012749260,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.715,2002-09-21,true,821593716.826,21:44:19.40 +940364413,37154931229302392,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.715,1963-08-06,false,259954074.874,17:14:25.49 +-539106001,52482689631861816,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.715,1955-03-07,false,750219206.355,1:39:25.24 +-1444333425,18700893103045316,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.715,1988-01-27,true,161351299.276,9:9:35.48 +-1628183892,5117021434790615,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.715,1970-05-14,true,19967246.5547,16:46:33.46 +-1549845130,17076558889090356,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.715,1954-08-11,true,71991165.4428,5:12:18.44 +500627521,12085134318433526,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.715,1982-06-27,false,1277085387.35,3:35:57.48 +1555822391,22807387958891624,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.716,2005-09-12,false,24691823.3871,2:29:32.18 +-1932866114,50354284831056368,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.716,1983-01-27,true,386924981.046,14:50:44.49 +-904768541,34353271177850420,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.716,1984-08-01,false,477008981.615,12:42:31.34 +-517154172,53555006184157688,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.716,1952-05-17,true,1108236485.48,21:58:48.29 +489082397,65472677375876784,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.716,2007-08-19,true,39847012.6396,14:22:55.32 +-795552518,15015208613974538,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.716,1979-07-26,true,386166867.523,20:27:28.30 +-1871640341,65739142566472488,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.716,2003-01-23,false,456408630.555,22:4:13.42 +13691484,43012605855189768,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.716,1997-02-18,false,100609374.253,10:32:50.52 +-999580449,58274958253810712,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.717,1996-08-07,true,1155993225.51,8:58:53.36 +902158694,85587419649582240,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.717,1950-11-12,false,672939493.539,5:50:46.8 +1977440352,7437532977747958,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.717,1950-09-04,false,895201061.27,9:56:55.39 +-378927253,88719449651812496,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.717,2012-10-26,false,1246005065.08,10:9:20.16 +-1524989343,4460835188960880,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.717,2004-04-12,true,566796848.815,5:55:9.49 +402375169,36097835921222728,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.717,2005-01-12,false,263485486.293,19:17:43.3 +-1626722927,25764018431689532,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.717,1954-02-18,true,653544558.077,7:35:58.56 +-535468611,1164853064247255,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.718,1984-11-09,false,558029917.13,15:3:11.36 +1054579705,16294035691340390,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.718,2006-03-24,true,527808599.889,18:25:17.24 +-133466485,80354749388942432,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.718,1984-06-23,false,607841760.329,20:32:39.49 +-1296589845,49592698657186960,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.718,1969-07-25,true,374237418.674,11:31:14.25 +-782656425,91904913943489152,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.718,2004-06-27,true,237733058.083,14:21:43.27 +-986578881,14751567023871826,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.718,1975-05-03,true,155443903.127,9:4:36.48 +1835175259,82731601555175552,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.718,1951-04-09,true,574514648.331,14:32:58.52 +1160640667,4942217307266294,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.718,1970-10-24,true,894419706.225,18:20:35.37 +608751422,87944958342318496,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.718,1974-06-27,false,50061364.7705,11:45:46.41 +-1775837688,58010223692828368,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.719,1986-05-20,false,510603850.106,20:40:37.10 +-522313522,71392848594631192,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.719,1974-04-09,false,588479722.169,17:56:9.10 +-1138953778,15088461163623292,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.719,1956-11-16,true,1084876692.76,4:24:19.4 +-1832334096,81916546186092224,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.719,1988-03-25,true,644009498.304,14:35:33.11 +-617663175,61827239029755248,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.719,2005-05-07,false,900645855.395,13:21:28.28 +-406872382,47956076405065680,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.719,1976-06-20,true,511434992.893,19:11:27.25 +-1579799178,53971722577789072,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.719,1960-11-03,true,710222660.875,18:58:27.54 +-1927451278,39748121027249072,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.719,2012-04-22,false,353156098.884,3:49:47.57 +-1666230438,47675186649110824,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.720,1983-03-04,true,1025337327.48,14:28:39.50 +1753990331,3519645684252334,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.720,2007-11-11,false,581935349.129,19:39:30.41 +2135955106,62880479871109248,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.720,1977-06-19,true,1262504842.0,4:56:10.24 +1204811368,71350169637837552,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.720,1976-10-24,true,8214843.35883,22:24:13.51 +-885711502,82205647507559536,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.720,2004-09-04,false,428683550.168,21:47:41.9 +-1999486273,90356217076455072,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.720,1991-06-09,true,475545952.705,22:8:47.12 +-2145817579,29827587126599372,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.720,1966-08-09,false,683077528.27,3:35:28.32 +-1056084048,28742238655413464,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.721,1965-02-09,false,528092764.203,2:35:4.30 +848227304,18235860552032992,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.721,1971-09-20,false,223904364.448,15:24:49.8 +2125893543,30382937776691356,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.721,1993-05-17,true,121011541.153,13:6:5.33 +-652017728,15007273885445940,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.721,1990-02-19,false,633357097.647,10:12:28.30 +1430966024,45357506980398272,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.721,1984-08-14,false,1012847901.68,18:36:15.12 +-2048707779,20126446280834232,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.721,1978-07-05,true,144875225.107,2:16:50.56 +1579275038,75317411448343936,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.721,1990-01-09,true,1284029370.95,7:52:53.50 +-1453879312,53465319907295296,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.721,1981-03-03,true,942849279.642,13:38:52.15 +2097120177,57682075834103336,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.722,1985-04-03,false,280585804.391,14:31:47.3 +-1945917659,82840753254775728,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.722,1989-07-22,true,842699028.611,22:44:34.47 +-1081146558,16537188918924002,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.722,1962-09-09,false,629878669.147,8:33:56.23 +623162092,83267493639343504,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.722,2010-09-10,true,1295616219.52,6:18:54.58 +283898608,9984425136805602,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.722,1953-01-16,true,668440405.573,11:1:17.56 +-1046248896,70416776091200824,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.722,1991-10-05,true,1320170703.2,14:43:47.7 +1468055197,7994205364507699,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.722,1980-05-21,true,559711294.525,14:25:45.36 +18105458,74232879802115984,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.723,1990-06-16,false,516854662.109,21:30:34.58 +-2054072142,88162045394428176,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.723,1988-02-18,false,283546625.714,17:15:53.8 +-2084890771,18144315445647328,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.723,1960-07-06,true,1144025254.67,13:21:30.42 +-1310231652,80526935648284608,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.723,1987-01-01,false,43546926.7378,8:53:35.15 +-1883865587,52619013199715736,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.723,2006-10-18,false,145520326.964,22:28:54.37 +-389752713,16315558219360788,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.723,1957-02-13,true,539463807.84,10:13:52.7 +-1669065490,204238306662635,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.723,2002-01-21,false,878504869.81,8:1:31.19 +-1657013936,89611506276237568,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.724,1982-10-15,false,46120775.2268,6:6:47.40 +-1541830166,80653747282828160,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.724,1960-04-27,true,563504214.978,7:22:52.12 +-841031342,55904729202954008,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.724,1999-02-19,true,1181665545.82,10:29:29.12 +-392925096,49436388286327880,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.724,1952-09-15,true,1269084088.42,2:44:51.53 +736718474,89411317918166528,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.724,1951-03-02,true,1210242187.09,7:8:36.18 +1634237912,33442058530793216,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.724,1969-06-25,false,140904030.429,4:55:54.53 +1064517198,2115756509532897,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.724,2004-03-21,true,495429484.162,9:16:39.23 +-1068976934,57647398393231752,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.725,1950-08-23,true,1119585021.17,10:45:34.5 +-914965492,27544293607123732,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.725,1995-11-25,true,113838506.614,13:34:43.7 +1913499574,71567238096158520,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.725,1951-10-17,true,1007332963.84,15:49:2.26 +-757811368,85261071200834240,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.725,1994-03-26,false,79361172.9378,11:41:8.14 +-1099450999,65203986862835024,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.725,1988-02-27,true,912919135.283,7:27:53.22 +760882340,18030338759382496,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.725,1971-10-02,false,119727521.578,3:45:53.36 +47162837,36755911289710592,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.725,1970-11-17,true,1225110008.8,19:2:52.26 +-1444544471,50405238055119040,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.726,1990-10-16,false,885816697.809,21:29:7.9 +-1007800013,23036794059831492,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.726,1993-09-01,true,442295215.864,12:17:14.19 +1493678588,57286895135823416,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.726,1986-02-21,true,1322938261.26,20:51:50.9 +1784356916,43339196805567304,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.726,1973-03-12,true,688910122.834,6:27:9.46 +-188987091,54813312017095968,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.726,1958-11-07,false,294187695.04,2:44:27.6 +1890978383,57916313182951080,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.726,1952-01-13,true,102416932.947,9:48:2.57 +-444818883,53078985633424264,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.726,2007-07-23,false,1316241640.79,18:4:49.25 +708249350,32873918584766280,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.727,1986-09-12,true,1022345060.42,1:54:53.20 +1606686375,9728708637987942,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.727,1965-03-01,false,55161208.7936,15:58:56.38 +1938741942,87559018923452464,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.727,1997-06-14,true,648182399.365,20:28:58.56 +1171822543,31389145649567552,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.727,1984-10-08,false,1368904134.85,12:58:18.50 +548321285,91949312758717504,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.727,1960-01-27,true,116604783.485,8:29:42.15 +1781890850,5513112827319378,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.727,1985-03-02,false,415970901.245,17:29:43.38 +-135365327,85404912436383600,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.727,2009-10-13,false,1270531517.17,14:7:50.33 +-948912023,37624548677386688,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.728,2003-05-26,false,624155192.911,16:9:44.43 +-1405729715,67561212250857088,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.728,1978-03-23,true,68881355.8197,20:58:38.50 +-1246116007,54927322691310768,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.728,1977-09-09,false,653296283.873,3:55:35.14 +-919236349,1967965555962675,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.728,1989-09-07,false,1128275057.66,15:35:52.16 +1063493918,28831436896690832,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.728,1974-04-23,true,1026986757.19,8:33:34.44 +496234618,37827266392666616,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.728,2000-05-15,true,1248035802.57,3:44:13.50 +1480065734,13160057061518388,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.728,1998-09-18,true,184581796.376,18:15:46.29 +541898797,76338969180306144,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.729,2000-04-27,false,1366544895.33,10:52:41.4 +265486505,31610912496646052,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.729,1973-11-16,true,941095668.983,8:40:24.18 +314407479,16181647193279786,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.729,1988-07-13,false,961471541.092,13:41:16.54 +-374992788,25311133448195164,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.729,1995-10-21,false,1121441192.24,2:11:11.7 +-951570243,88900919737336512,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.729,1989-06-09,false,787611918.687,17:20:35.48 +-1337934351,24722358179821652,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.729,2000-06-17,true,582215363.798,11:15:33.31 +-1816811714,42768049104409384,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.729,1993-01-14,false,742312077.296,13:25:38.53 +822020258,54123983296790192,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.730,2004-09-10,true,971597011.724,19:29:49.39 +142295962,551718702451527,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.730,1984-07-09,false,1277386634.82,16:48:5.17 +-695893408,22582675240700396,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.730,2011-08-08,false,1093117657.25,2:10:20.10 +-1107618067,65589253076918552,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.730,1990-04-06,true,1291614027.07,15:26:38.2 +1851295391,8819412953320551,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.730,1950-09-19,true,595827848.513,21:30:26.50 +1719135820,77008143651986704,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.730,1957-10-10,false,116227884.759,19:53:40.45 +1625535298,67534935560882104,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.730,1971-09-17,true,233082754.41,3:22:53.26 +-641237157,61548236981958432,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.730,1976-10-06,false,135484445.649,6:43:6.58 +-651694823,23038217011015088,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.731,1981-04-16,true,483146765.946,18:5:7.22 +-1012293527,7114587064045947,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.731,1964-02-24,true,751682579.816,4:32:43.40 +799760159,43185061583861112,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.731,1964-04-21,true,817832531.13,9:53:54.49 +-1996341756,17565399626192026,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.731,1994-05-16,false,566541214.964,15:5:27.31 +-522291464,69158749676330368,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.731,1966-09-01,false,681228805.947,6:45:10.27 +-497344150,74773924898247760,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.731,1975-02-18,true,36892017.1821,22:53:28.4 +-2051916228,61552307419031264,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.731,1977-04-17,false,331879091.268,18:34:55.49 +140163868,85343440598143424,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.732,2002-03-12,false,1240853557.8,12:12:27.50 +1143159696,81893672038364768,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.732,1966-11-26,true,1211898264.48,12:48:7.56 +1511909531,76928462798488864,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.732,1979-10-06,false,473191487.533,1:20:27.16 +-689888707,34986868854287708,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.732,1974-06-21,true,319744507.579,17:55:49.19 +-1037382670,13820919028306094,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.732,2007-09-11,true,349911038.741,14:11:9.30 +1233783905,72031323719893584,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.732,1963-03-14,true,983038916.561,17:18:16.7 +1166234769,86784336862750624,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.732,1957-05-08,true,1317694065.7,21:58:20.5 +-897835386,22564975491929968,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.733,1956-03-27,true,207915374.424,8:9:14.25 +2017803303,85304718003825776,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.733,1981-05-09,true,619418162.667,8:48:35.4 +688198932,83756747647997104,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.733,1966-09-01,true,779050626.995,4:28:47.16 +-1958792620,11714965661942150,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.733,1955-01-17,false,357647204.743,6:50:10.6 +-1810942308,87014461241207008,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.733,1985-02-18,true,1356489119.85,11:6:17.39 +683975285,8113162530952346,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.733,1956-05-16,true,325966363.416,13:3:25.24 +-730960511,76438540149079024,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.733,1971-02-01,true,256059175.055,1:24:44.4 +786400326,27756539989472248,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.734,2012-05-01,false,247306615.152,12:19:30.18 +187021800,65734105263329488,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.734,1998-02-25,false,1388743114.99,17:3:23.55 +-793180186,24050504218549752,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.734,1991-03-01,true,427975579.232,12:23:1.37 +1564948477,48982597436955176,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.734,1998-07-14,false,811841454.225,4:35:46.29 +-1279495172,3189851023104686,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.734,1996-01-20,true,1274850548.31,7:4:41.29 +-1174432345,37447857888794952,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.734,1989-08-23,false,724846820.946,14:4:46.32 +-1146520881,58840013921022096,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.734,1990-08-23,false,272612714.298,13:3:6.17 +-537022055,7789526674221763,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.734,1966-04-05,false,563274889.274,20:23:8.55 +245797799,80809995007168624,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.735,2010-07-14,false,880538957.45,2:26:6.9 +-1297691197,31298320083573012,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.735,1977-09-19,true,1052523307.15,19:6:36.53 +978137346,58744186349742624,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.735,1960-08-15,false,1263290772.48,11:16:52.28 +932223351,75838665405534320,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.735,1959-09-09,true,49355906.0341,1:20:34.10 +2004585042,74376537433282768,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.735,1978-11-24,true,948296875.194,20:13:43.53 +1323681416,10888248072516628,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.735,2002-06-25,true,1430523951.37,11:16:15.29 +-1590667260,36944456306913328,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.735,1968-08-18,true,1196480491.7,5:15:42.26 +532007844,24273563282456004,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.736,1981-07-27,true,1328545215.8,15:21:27.21 +7418692,27863069698829352,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.736,1974-05-24,true,1250907793.74,14:44:38.34 +1312461508,15097942390808770,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.736,1965-02-18,false,1393904534.24,13:51:32.51 +1361187003,38042695595076272,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.736,1958-06-20,true,1186373024.15,3:34:26.13 +2100506553,23483345134115892,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.736,1999-10-17,false,945895630.444,17:38:53.26 +-1204579618,71027460028421032,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.736,1977-01-26,false,756890754.198,18:30:48.48 +-1278301603,69115498834752320,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.736,1965-08-20,false,1199705195.35,7:47:10.37 +-1070782858,12127480159488984,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.736,1992-05-20,true,686920756.992,11:27:9.51 +1528934513,47014048592177624,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.737,1982-04-21,true,931795026.717,1:31:26.26 +-1658426539,30975145175982468,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.737,2010-08-04,true,684534451.896,11:37:41.47 +81177858,50419892054900032,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.737,1988-07-23,true,343340027.526,22:8:7.28 +-570389468,40533920993118280,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.737,1960-04-27,false,58400032.7821,3:8:15.9 +-1731244619,87686091363087264,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.737,1954-10-06,true,624341508.108,20:3:51.48 +-1941124479,62064288069261864,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.737,1979-08-12,true,1407688311.92,16:40:11.43 +1208649942,23811731421333196,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.737,2003-01-06,false,894946445.45,7:8:24.1 +-1307750838,58010414908936440,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.738,1959-11-07,false,12683734.7753,7:27:26.9 +-517066153,31061220510216244,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.738,1964-01-15,false,1101084219.32,7:12:26.20 +-242117016,22471900293919344,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.738,1960-01-18,false,1013608415.54,12:24:38.26 +-1463642000,57597311481751376,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.738,2006-05-16,true,1401025747.98,10:16:49.35 +1968194733,70039958050456832,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.738,1989-03-21,true,664791579.395,20:17:44.22 +-1454608858,59211241779913168,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.738,1969-09-26,false,322662460.557,4:40:34.48 +104407856,35270305474827572,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.738,1993-10-01,true,1362014733.84,22:45:43.46 +-117894725,68369678726993360,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.739,1966-07-23,true,879259863.822,17:57:8.8 +-1882853886,85139796276843136,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.739,1965-08-09,true,886645052.936,11:33:21.42 +2057080807,30924116352261316,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.739,1968-03-13,false,490466829.462,12:31:16.7 +2028324612,54833029030498376,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.739,1983-04-26,false,629406298.821,6:51:5.37 +-1847725802,62036924594676536,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.739,1977-07-12,false,711791620.245,5:2:29.54 +2050161551,91072032398579616,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.739,1973-03-17,false,1236693402.83,4:40:35.16 +-1828472322,26494613815227176,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.739,2009-09-21,true,604865055.23,10:33:33.56 +2127614999,69228645162511048,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.740,1986-10-14,false,1075241875.61,4:4:22.54 +212761027,3427104861518428,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.740,1989-05-20,false,721702316.188,2:32:22.47 +-643053148,60245007045352832,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.740,1952-11-03,false,1150463444.82,17:23:35.46 +-1664167115,1638503009247529,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.740,1953-11-04,true,405072991.049,9:29:30.29 +736846514,11532581887910830,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.740,1966-04-07,false,344612692.479,10:39:6.46 +1356994056,45037868713879728,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.740,1999-08-10,false,956712704.694,19:54:26.37 +803901897,75358685682613312,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.740,1964-11-02,false,1203631349.62,22:43:38.2 +-1458812294,30383949446981264,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.740,1990-10-22,true,964127767.3,5:14:21.32 +1284451582,16375932180664372,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.741,1997-02-13,true,877536501.021,1:49:36.2 +817872900,12684426474072566,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.741,1950-03-14,false,1204257070.11,4:23:13.36 +1106665690,4949662025772575,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.741,1989-07-13,false,7318769.15634,22:42:5.42 +-1964837954,19579875139048212,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.741,1962-09-16,false,757171962.664,14:26:11.40 +244512868,4559846026366218,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.741,2004-06-12,true,135809426.577,18:8:34.43 +1817035888,86336991378152784,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.741,1973-02-20,false,350579311.55,2:12:12.12 +1993502025,88530941433307888,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.741,1951-01-13,true,155706641.338,16:21:42.56 +462644917,84330503701193344,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.742,2011-01-09,false,1044906549.63,6:51:47.10 +1002771478,53970830370467320,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.742,1953-10-05,true,1809820.36683,3:29:27.4 +-594475413,37152225088835536,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.742,2005-09-27,false,1089028417.32,3:55:8.20 +-978787024,37886762045209968,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.742,1990-05-21,false,429688615.78,7:23:40.14 +163168678,70921173878788008,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.742,1995-01-09,false,649366078.811,14:45:33.4 +679463040,49020878530610344,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.742,1980-03-26,false,396047017.266,11:51:54.34 +-681567307,76053684011934400,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.742,1987-05-14,true,848479053.919,13:36:38.30 +-1307484180,40862549148310624,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.743,1966-06-07,true,1372522502.13,14:2:53.58 +780967805,7025843850857626,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.743,1953-04-11,false,10608194.952,11:53:45.30 +2064725156,28854141112639344,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.743,2003-06-08,false,1087517261.51,3:51:11.9 +-1715217044,33576244147498292,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.743,1986-05-27,false,907938938.747,15:43:36.53 +1285585914,92066196042034496,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.743,1960-04-04,false,1160518408.0,7:36:37.56 +-1445878737,13075939278671186,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.743,1999-01-16,true,482554139.726,8:6:5.28 +1834039473,63966573140344064,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.743,2010-11-14,false,893490732.615,12:43:6.8 +-380608386,28459682961480416,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.744,2010-05-14,true,335811549.366,12:33:12.19 +-2035635619,20665093943913860,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.744,1997-07-04,true,461439853.314,11:6:50.19 +-156591403,46089327201602056,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.744,1981-03-17,true,1267158666.86,20:51:44.35 +-1631483642,14501781963945258,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.744,1967-05-19,false,205097371.139,14:7:4.16 +-1190405982,9177755678720840,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.744,1966-01-01,false,307233544.954,7:12:28.24 +1004757955,59546390699013312,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.744,1956-02-07,true,58340359.7977,10:25:51.26 +-821115946,41711324979002168,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.744,2004-04-08,true,502933156.924,18:45:6.43 +-194001228,37464802689466024,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.745,1997-06-02,false,937453756.111,4:40:17.55 +-412302074,91332168704524320,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.745,2013-04-06,false,369226546.62,10:40:5.10 +943180339,49070199148433840,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.745,2009-05-02,false,82615565.3272,3:56:11.36 +-1060746754,13720924486455470,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.745,1992-04-04,true,1091587846.96,7:40:7.54 +408022596,25968143291394324,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.745,1999-06-23,true,638747049.959,4:41:53.36 +-456492257,61472802353319408,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.745,1990-02-11,false,1336484801.8,4:42:46.3 +-48560758,14319417282686946,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.745,1958-04-19,false,688810078.187,17:52:51.20 +-1012082728,24516236032030628,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.746,1976-05-05,true,853167085.017,3:50:41.51 +-1606176269,19411065969357344,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.746,1974-05-15,false,1207161089.35,16:45:58.48 +-604860814,51289614993493744,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.746,1996-05-12,true,120987416.398,17:1:35.16 +-375300530,30953287083402640,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.746,1975-08-25,false,1330831789.21,4:37:49.29 +-639703616,56314573588418616,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.746,2001-10-21,true,81718232.186,7:55:51.5 +-697215789,27398621545210060,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.746,1956-05-08,true,1017312414.24,6:29:34.6 +-10542478,85336413326174176,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.747,1981-07-23,false,195700673.462,13:22:49.30 +-1528563080,32544435868322572,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.747,1998-03-05,false,980280240.767,11:25:46.2 +1157857693,67432599115957056,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.747,1950-10-07,false,93937475.7577,14:44:1.34 +1769565318,73301412638486656,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.747,2002-07-24,false,1345033750.38,10:10:35.9 +-485579550,7161190587258942,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.747,1973-11-12,false,1296894951.2,5:10:9.13 +1122794185,75642682173656096,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.747,1980-08-24,false,495758888.145,19:8:28.44 +2039629162,32386203959299136,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.747,1981-10-26,true,867855320.387,10:5:22.6 +398140085,75933540775669984,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.748,1967-03-09,true,576870431.268,10:2:53.56 +-1390708007,64369191765922088,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.748,1962-07-09,false,924534590.463,20:51:11.13 +439030740,4387509793719439,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.748,1987-07-10,true,420001908.557,21:2:41.28 +552536361,88298985335476656,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.748,1961-03-23,false,1154488453.28,10:4:29.41 +-570736065,36641026448013264,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.748,1985-11-09,false,1068440486.05,9:55:49.21 +1420298141,1393607696570275,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.748,1995-02-26,true,1261781477.78,20:11:21.37 +-21598837,71456469937339320,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.748,1952-06-26,false,8704425.43077,7:4:2.13 +1420354729,72522367043925616,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.748,1975-03-14,true,822367318.059,13:26:54.55 +1128628350,66844302515541328,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.749,1979-02-26,false,273325088.402,12:16:31.13 +394394608,39136850125297528,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.749,2004-04-26,false,714541666.855,21:56:47.46 +1477466408,84619120771217888,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.749,1978-02-06,false,315366635.543,7:36:56.55 +-140911125,22205118770125240,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.749,1953-01-02,true,1007950139.26,9:48:10.34 +-60981787,40600409463508032,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.749,2007-07-20,false,911891590.072,21:17:37.2 +-1295465997,88807511638259824,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.749,1970-08-20,false,1180727111.41,16:36:37.21 +-1593294532,58610003891718824,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.749,1957-10-24,true,878439117.704,6:48:32.47 +1470753145,63067502912237352,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.750,2010-09-23,true,1035055490.56,3:18:46.23 +918795047,53194120955593640,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.750,1982-08-11,false,117289271.662,21:20:7.18 +-597018913,85094682438762496,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.750,1952-11-06,false,1071020478.32,18:7:22.15 +-53272358,59556683061389480,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.750,2011-06-17,false,430678292.012,3:27:39.39 +2095822207,58222221580132208,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.750,2012-10-27,true,700730835.195,10:22:14.48 +572133867,27212699687757376,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.750,1991-07-18,false,1357764522.57,21:52:33.9 +-840455538,62696915032543072,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.750,1994-07-19,true,429424094.668,20:29:22.25 +-507722766,6148418863653458,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.751,2005-09-01,true,859108266.532,1:32:39.35 +-1934255422,58243863473964472,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.751,1989-10-12,true,433242397.7,9:56:32.39 +-578117335,1102003138132940,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.751,2004-10-14,true,754470522.63,11:36:24.12 +1406359407,52164549701053880,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.751,2010-01-02,false,1365728595.02,1:54:18.27 +-1621919174,21481887849070336,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.751,1967-08-15,true,1161360643.3,3:27:50.42 +-914576980,31707844479062548,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.751,2001-04-22,false,669555652.885,19:11:4.34 +-1005387202,4026860555642122,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.751,1989-05-03,false,1069274729.97,19:30:31.22 +1403407452,17133343879200962,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.752,1987-07-15,false,172232517.602,9:13:52.20 +-1000786859,83835874415234128,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.752,1973-03-09,false,287832534.17,13:42:25.41 +-1412565164,36693637227993384,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.752,2002-02-04,false,677689022.255,21:18:33.16 +376630199,31449897784814448,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.752,1971-09-07,true,820821303.708,20:54:55.36 +-569269507,66547210184836128,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.752,1977-06-15,false,1007257003.64,6:45:26.4 +1677815584,27105740117679760,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.752,1978-01-10,true,866389229.005,21:37:54.20 +-291398406,1122513907872952,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.752,1970-06-24,true,1015194652.49,13:10:9.17 +897508063,15925255745818676,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.752,2002-01-06,true,421755227.815,9:35:56.33 +449321897,40472645164473296,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.753,2005-09-26,false,792061327.539,20:10:10.2 +-1207810613,20611401831405024,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.753,2013-03-16,false,452494143.684,14:23:23.49 +-562363406,91929084317637504,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.753,1979-09-23,false,514223811.821,6:37:14.19 +-1136437798,72346437148061696,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.753,1972-11-06,true,547896328.43,18:24:44.52 +-1336180663,23023685982218988,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.753,1984-02-20,false,29397378.3742,9:29:14.11 +-1420573263,76097008111695728,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.753,1966-08-08,true,1139035830.95,15:15:24.50 +-1300903296,31110795474429420,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.753,1968-05-04,true,1366355013.6,5:9:57.40 +-1481441571,28190314517589240,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.754,1982-07-27,false,174034891.22,9:32:36.3 +1925248267,17587969941692704,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.754,1999-06-05,false,59265091.5353,13:46:48.13 +1287566943,57689528860981640,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.754,1976-07-15,false,1184062449.44,21:49:14.42 +-552953668,85429656637969360,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.754,1970-05-09,false,637657880.14,15:51:28.26 +-753151785,74126894311347184,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.754,1990-08-15,true,1008494896.22,8:37:57.40 +1367149342,35024506150712524,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.754,1996-08-03,false,194730276.398,2:26:52.54 +2105031369,79616142557686496,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.754,1980-08-27,false,1354953054.46,9:33:12.25 +-929545677,71918526102366352,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.755,1976-10-05,true,45508073.7634,14:27:19.15 +-79542113,56869415199552616,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.755,1964-04-24,false,190543214.244,6:49:7.48 +1967614570,68164835426111304,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.755,1985-01-17,true,665879007.142,19:20:5.42 +-2085023400,70919885553245328,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.755,2010-07-13,false,1156817313.17,12:52:51.57 +-360480217,38192760502966608,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.755,1988-04-24,false,1367673829.86,20:40:2.39 +1163876050,57747674008785696,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.755,1974-01-14,true,1203561634.3,9:30:22.42 +-192130595,44364040529866928,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.755,1957-09-17,false,121369542.662,13:51:40.26 +1183122672,85799310099860000,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.755,1975-10-14,true,518952464.921,22:21:36.4 +-847477771,40939837224815480,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.756,1981-08-01,true,1161454498.01,10:41:6.46 +-1230747752,85781992124426224,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.756,1962-06-13,true,976145726.636,21:25:16.48 +1061655440,2016570527464704,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.756,1991-06-08,true,418716158.878,20:28:29.26 +1506219630,70318419781963320,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.756,1966-06-13,false,984560924.461,4:7:1.37 +45931741,84614216907775760,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.756,1986-01-14,true,1046089148.52,12:19:21.33 +453083181,52698300092685856,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.756,1964-02-16,false,716013594.885,5:22:15.39 +474327337,31412237072146352,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.756,1967-07-02,true,1005387834.05,17:5:16.5 +1924866486,28225232124945492,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.757,1991-04-18,false,1335052229.34,17:30:22.42 +-314706216,38285104841508928,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.757,1966-08-22,true,1360305224.28,19:28:30.52 +1354783888,59560086273806144,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.757,2002-11-07,false,831831330.134,4:54:18.25 +-1148765658,68404499487132888,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.757,1986-08-06,true,929676677.421,17:16:52.6 +1821901091,39781042792529320,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.757,1994-08-17,false,162582268.75,13:24:31.50 +-234524649,86478644132356912,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.757,1984-03-08,false,160634009.375,8:36:37.46 +-1250387505,59233433992303648,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.757,1971-06-06,true,787500126.686,7:34:39.26 +-1443637381,30090711773944004,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.758,1979-01-06,true,747200855.83,6:10:21.41 +-1310801523,59703318786980960,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.758,1973-04-23,false,1252289041.4,15:54:31.34 +-528394582,80837002210622192,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.758,1996-07-15,false,128991842.731,7:17:17.14 +-302194191,32430494938675868,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.758,1996-01-19,false,612657812.081,12:48:57.26 +-1165005624,60686760984754368,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.758,1956-05-10,true,359613525.706,7:4:7.54 +-1983378621,3729648332193853,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.758,1966-11-23,true,20491191.704,19:11:50.29 +738588848,61013136922983664,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.758,1986-01-19,false,192666041.495,20:30:18.26 +280209261,44375387022743576,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.759,1956-04-14,true,155982293.169,22:9:11.56 +-1698606624,49593153671395336,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.759,1980-11-16,false,533514396.665,16:5:38.54 +-410850713,88142271195151936,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.759,1976-06-24,true,1393869081.51,22:7:24.57 +-1552674373,14452919536434474,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.759,1969-05-24,true,789244568.442,15:50:26.10 +-1287609448,16927143345732884,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.759,1968-05-10,true,218013174.74,7:7:56.45 +-1347211376,77321952358370400,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.759,1975-10-26,true,874189596.229,19:6:13.5 +-210630428,64684533859031304,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.759,2012-11-12,true,336179202.193,15:8:2.6 +1030446554,27542494674651372,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.760,2010-03-13,false,237411512.836,1:9:50.32 +1499043198,88763043899717104,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.760,2006-03-10,true,496673451.24,9:34:34.18 +-1341700457,2885575446348605,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.760,1959-03-17,true,739271700.749,17:58:27.10 +-728485630,57889702024915904,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.760,1969-07-01,true,490254434.396,20:21:54.34 +-2068507692,55266212819521648,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.760,1974-06-07,false,593618672.962,5:34:22.49 +-319878499,39989444489365176,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.760,1992-03-05,true,1430133009.87,18:2:54.22 +-934372536,44491507693895848,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.760,2003-01-22,false,35969928.718,20:28:37.13 +473902018,76019600366916896,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.760,1951-03-26,false,1183916410.1,7:11:22.27 +1922493429,24038050042813132,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.761,2012-11-17,true,922931815.939,5:2:8.53 +-1116522147,10516582881467556,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.761,1972-02-13,false,906120508.635,12:39:12.51 +470860383,46377751968000248,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.761,1973-11-12,false,317354125.247,5:55:18.42 +-1926172776,85824132098275536,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.761,1978-01-06,false,1077072234.74,5:13:20.18 +-172442334,73980876946125728,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.761,1972-03-24,false,62518227.8699,14:45:7.48 +-1799244580,14420895285281618,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.761,1966-04-19,true,882889979.605,13:38:30.23 +2090116522,41209092586330048,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.761,1986-09-20,true,1201625968.54,15:30:45.15 +1082034314,54649197096009976,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.762,1976-05-23,true,1259017420.33,18:40:6.9 +-419989280,63134803798832512,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.762,1975-05-05,false,558672554.645,9:54:13.18 +1214957620,2058678960695224,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.762,1991-02-01,false,1133802022.75,14:16:28.29 +-785532196,83531763178526464,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.762,1987-06-08,true,1346816659.8,7:39:12.56 +-1120057457,87772468544426624,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.762,1999-04-19,false,576120311.865,16:19:4.44 +780917214,65281058041463792,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.762,1998-09-15,false,598634161.953,18:36:27.2 +-871372859,53346149872891304,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.762,1956-02-13,false,1107757860.9,2:4:55.41 +-390277510,16315791656664392,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.763,1967-07-09,false,1138374759.06,11:32:18.36 +551053571,77910853553071648,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.763,2007-02-13,false,756333533.454,14:37:57.25 +1382788602,86415060475541280,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.763,1964-05-15,true,1003971340.05,3:4:43.42 +-1853609467,79806636036796768,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.763,1950-09-16,true,802713778.295,6:46:53.57 +-994755281,18647067814941452,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.763,1960-11-26,false,323128359.482,12:3:9.6 +1273360619,83634877883796608,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.763,1984-05-18,true,846823852.722,12:36:36.52 +1566646738,74928415369142240,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.763,2002-07-07,true,1085831348.51,19:14:7.39 +917505566,18400545988858092,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.764,2004-01-07,false,669201392.992,15:30:26.20 +-797899634,52981729500545760,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.764,1981-06-27,true,386868197.256,21:56:6.21 +779854034,88992946890199600,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.764,1969-02-06,true,632875571.737,3:52:39.50 +-2081443961,18252917990156184,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.764,1997-01-15,false,1091483484.45,16:32:23.35 +680625506,4560227566708654,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.764,1963-09-02,false,447768660.407,12:55:27.8 +-1933335786,20915291350354700,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.764,1972-01-21,true,706744209.446,6:15:50.47 +-1935065590,32081250582465620,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.764,2010-09-14,false,1024020336.93,14:2:31.26 +656482249,85892630895198224,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.765,2007-06-13,true,449356557.869,12:24:54.42 +-544976263,1711140498802268,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.765,1956-02-24,false,1135261366.78,6:24:30.42 +-1449068591,5544564875456840,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.765,2013-08-21,true,76818355.238,17:40:54.57 +-156353123,72265556678339520,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.765,1958-04-17,false,131205705.99,12:1:53.34 +2036857985,37354982477185288,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.765,1997-05-21,false,104439837.086,12:45:1.44 +-1318369840,79328424977956736,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.765,1970-01-15,false,343300256.531,14:9:30.21 +-1454323628,1813277393676708,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.765,2001-01-27,false,1156384159.14,12:24:25.40 +-912198433,39854444100181768,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.765,1997-08-01,false,61383260.2196,21:17:49.50 +186499928,43708202020203344,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.766,2013-09-11,true,1059518209.19,10:13:20.36 +-1083488497,77187717484950432,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.766,1988-04-10,false,630725932.135,10:40:33.13 +751300642,14110096622282876,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.766,1956-08-06,true,276087224.793,11:33:4.13 +1537580534,31834113056452568,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.766,1959-04-23,true,660508455.913,21:19:54.20 +302388770,84635162325424256,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.766,1965-05-22,true,499429386.792,9:33:38.4 +-852722131,31464355431464796,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.766,1979-11-04,true,747371813.292,20:36:8.18 +-2003571142,85793855648191232,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.766,2006-06-06,true,256613998.415,12:11:41.56 +1089995817,85775246034642032,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.767,1987-08-16,true,1292266729.66,2:22:32.36 +-524692112,48692661853943320,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.767,1996-04-24,false,704453690.09,17:57:24.28 +-1697413769,27091128004088688,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.767,1952-05-22,true,921645855.262,18:20:36.28 +-515752820,19451294542937404,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.767,1971-04-18,false,1422210683.33,20:49:15.46 +-1741018433,72676299876550928,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.767,1956-08-19,true,217395221.395,9:41:21.32 +-572370319,29292089372334888,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.767,1968-02-12,true,1412471899.9,17:27:15.8 +-1022067392,46659566774728824,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.767,2005-09-17,true,1022772343.4,3:35:16.21 +2068015756,17926499188113112,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.768,1977-03-22,true,1188490098.63,4:27:52.27 +478173410,43732664931243288,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.768,1953-02-19,true,182306655.04,19:23:58.45 +-2091409279,7988457155893361,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.768,1986-10-14,true,817924921.152,18:18:19.48 +-947540059,90193284452536976,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.768,1984-11-13,false,109008023.407,8:37:17.15 +1542860684,35344291017922796,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.768,2013-08-18,false,243588500.295,17:3:33.22 +1871324717,6648524722352097,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.768,1993-07-15,false,412246187.406,2:51:7.27 +1731094110,12419208562447616,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.768,1975-05-04,false,44307182.0488,16:21:48.37 +1362885670,39666706750065648,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.769,1975-04-14,true,156172558.278,7:13:13.14 +-53439150,41191292714353856,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.769,1954-02-08,true,1427561654.53,3:17:16.44 +241746812,5230686804315699,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.769,1971-07-10,true,761956303.261,21:10:4.20 +1900205555,22773653971878092,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.769,2010-07-05,false,1191442756.54,22:50:53.52 +1345473473,62615129705190224,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.769,1992-10-20,true,688288733.161,20:48:22.43 +-718236583,6126858158742835,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.769,1987-07-09,false,497079931.056,5:38:13.26 +997239615,18936575998789396,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.769,1955-05-02,false,26460761.8579,21:19:17.12 +-1736994671,43866467917725000,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.770,1983-09-19,false,540502902.17,5:49:25.14 +-316417036,46863195367794928,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.770,1999-11-22,false,648253567.872,3:35:49.19 +-333979134,45095952909367152,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.770,1966-06-21,false,605071372.335,5:12:15.44 +1563714334,14773029007838670,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.770,1981-04-09,true,1270685714.62,14:4:45.51 +-731802467,66375595271747720,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.770,1951-06-23,true,616497487.364,9:1:41.56 +-1145032959,27050714543276452,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.770,1956-04-06,true,37790141.5805,7:28:11.56 +1650929125,15607625054143110,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.770,1957-11-26,true,461846170.699,21:41:58.6 +-1916688277,61585353106975992,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.771,1987-01-23,false,918444232.233,2:9:54.23 +-1447171646,63275664259509384,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.771,1974-11-11,true,815997599.058,19:11:58.11 +491144513,65025085562686816,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.771,2010-04-11,false,338964661.155,20:47:36.5 +1043808700,54852615125332576,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.771,1967-01-01,true,1432480278.67,8:49:4.9 +674052709,72251921191498064,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.771,1990-09-09,false,502450203.751,19:24:23.58 +-1622892217,22899923350710980,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.771,1966-06-10,true,1039801642.31,14:35:48.23 +846090436,75131768267295712,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.771,2013-06-25,true,609088496.295,15:7:1.28 +39714707,27914806471733728,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.771,1957-02-05,false,282959622.438,22:7:39.52 +1997145692,75275949601770240,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.772,1968-07-08,false,736164598.664,3:34:29.45 +-827561375,6485666968839670,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.772,2006-07-25,false,760394105.58,20:14:54.40 +-1909581094,1837502073099356,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.772,1960-02-27,true,778019173.156,5:34:16.16 +1360730954,81455163846991808,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.772,2000-03-26,true,189965604.63,22:12:9.34 +732911562,38252728631737088,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.772,1958-04-11,true,954598422.035,6:9:43.46 +889767808,67815071326743576,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.772,1973-08-26,true,1413098808.03,10:18:24.8 +-500716300,54885144905544032,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.772,1986-10-20,false,1218327848.82,1:8:37.5 +921278193,39200099307088096,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.773,1982-08-09,true,1086071159.67,6:30:37.35 +123624807,31197619105579612,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.773,1980-04-15,false,664409071.72,11:51:53.38 +-428690994,32701983553148224,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.773,1965-11-04,false,970060860.972,17:25:54.47 +-1682040172,37248714860624344,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.773,1993-09-08,false,1367481304.06,2:16:15.38 +192296472,2267385908106271,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.773,2007-08-05,true,1396377159.65,19:32:23.32 +2142927366,42545183307232936,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.773,1986-02-10,true,1167082527.14,16:2:58.25 +873582154,56153640278182112,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.773,1952-08-03,false,125135047.147,18:37:4.23 +-1955401194,11306301336338576,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.774,1967-05-03,true,384295962.743,14:56:55.5 +1312151036,13032329224454636,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.774,2010-04-01,true,363271038.34,13:44:12.17 +-17292762,68879206276235336,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.774,1955-08-03,true,892124090.188,1:30:54.45 +-1204930140,51583058861696296,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.774,1963-08-09,true,174078783.9,1:29:10.33 +1748245587,36552901390764664,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.774,2011-06-27,true,1234344191.09,1:14:42.1 +2087606598,3679856751550894,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.774,1955-04-12,false,24035110.1341,5:35:49.47 +1685376553,70976660151748856,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.774,1956-07-20,false,618400138.332,7:53:50.23 +-1715087259,81117971616488736,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.775,1961-03-17,true,517839758.142,19:30:46.57 +-105846315,48235636465016504,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.775,1977-03-23,false,1188377202.45,16:21:37.6 +71240893,40421527822596064,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.775,1985-10-05,true,576646187.266,4:33:17.49 +-1308896640,56772935071732776,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.775,1997-09-20,true,514456698.228,1:50:35.48 +-1860121490,79769202906088880,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.775,2001-06-22,true,982941932.911,2:3:55.38 +-929906981,38208370759226768,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.775,1988-09-15,true,967278434.196,20:31:38.26 +-2069402960,63246468331196984,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.775,1983-07-18,false,375227083.137,14:35:45.13 +1519161356,79539163080246544,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.776,1969-02-12,false,277110131.189,15:30:3.56 +-349299828,19446377722732800,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.776,1976-09-12,false,663506383.156,10:8:38.49 +824413887,65818447903381712,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.776,1997-10-04,false,1296855380.99,7:45:30.14 +-949131493,16877138010837024,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.776,2001-09-03,false,633670067.288,19:32:29.15 +2013285676,85027071267216192,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.776,1978-07-15,false,1164005205.62,5:31:47.34 +743528045,81777666236564656,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.776,2008-09-22,false,334660116.994,6:34:31.36 +-657875368,62741052747420888,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.776,2009-02-03,true,1168915128.51,11:28:50.20 +736025173,33233322278859336,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.777,1969-06-06,true,220834118.114,21:25:9.5 +2109101143,46428753681093088,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.777,1993-05-12,true,839182210.532,2:53:7.43 +-993699685,26886731401525984,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.777,2003-01-22,true,899279493.35,18:3:57.35 +1779593177,7753737043664210,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.777,1979-09-08,false,658273807.857,3:18:38.28 +2019243290,47552934842648456,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.777,1990-09-08,true,204381769.175,8:1:53.50 +-952859382,54359748147627888,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.777,1963-11-17,true,980828661.665,1:31:8.47 +2045905445,84706603177546896,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.777,1994-07-21,true,45422101.8228,21:54:17.42 +2130202015,84808764525619296,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.778,2003-04-01,false,906742963.268,13:8:3.23 +1356565311,59780123152899544,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.778,1954-09-15,true,588386592.076,2:19:36.19 +318757173,48074285933595656,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.778,1986-03-27,true,1192647768.25,5:54:49.9 +-2110899345,10964610814568848,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.778,1971-10-05,false,1329959054.71,2:20:20.26 +-758894659,83984360313828656,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.778,1994-09-20,true,1263850125.39,14:53:29.1 +-1740427603,57508614114126528,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.778,2011-07-19,false,1329966284.32,15:24:31.9 +-138590796,19533933504395204,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.778,1961-01-27,true,881949796.919,20:39:46.17 +-524932388,90994767041699584,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.779,1971-08-08,false,692651803.974,19:23:49.49 +1322447667,81302790082219136,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.779,1997-07-21,false,608701270.658,16:5:56.35 +-934525244,64341106347878288,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.779,1985-10-08,true,237581119.792,3:2:31.30 +842390895,84638183832994784,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.779,1979-10-06,true,612496318.45,3:44:47.57 +1091691285,40012471006210592,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.779,1977-08-02,true,1009461452.64,6:10:17.29 +774068661,83190509400878112,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.779,1977-10-16,true,1348619161.07,14:41:21.44 +-1562926074,70985912338695192,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.779,2011-09-23,false,546342395.165,15:52:16.31 +118211219,60902418219697064,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.780,2007-07-19,false,1171799111.24,11:12:57.39 +-2061436064,18682372276590624,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.780,1988-10-01,false,962022446.552,1:23:15.49 +-1061701914,67414767591519344,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.780,1956-06-25,true,442598829.555,22:39:41.16 +722571144,16735367380234854,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.780,2009-06-12,false,1265023864.07,21:6:44.36 +225395951,48875646434937840,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.780,1954-11-04,true,105045984.121,18:23:11.38 +388399814,52467958611985416,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.780,1955-11-15,true,155549050.417,15:35:5.24 +1837334521,41872087409253000,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.780,1971-03-01,true,402775747.258,10:40:10.10 +-1273108088,39356215459095704,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.781,1970-06-10,true,634146975.426,13:36:19.4 +1421368844,33356546898718812,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.781,1995-04-14,false,1261683157.85,21:29:21.35 +-1123470651,59333783699626568,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.781,1973-07-06,true,376702117.295,18:33:18.41 +2098002566,86957452429897728,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.781,1989-01-27,true,779672996.322,6:41:29.19 +-1422671433,58231068165945264,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.781,1969-02-13,true,59872485.6706,16:32:13.42 +-1702703351,10357293316772638,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.781,2001-02-27,false,1332110492.53,21:37:7.19 +-2126869701,12422321948227196,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.781,2003-02-05,true,1366484380.04,14:54:48.3 +1446608088,87178223372673760,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.782,1991-03-20,false,14797942.6084,18:47:37.18 +-516202752,83864887612348256,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.782,1971-04-13,true,1261077612.76,12:17:37.3 +-1722999573,49225368553011112,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.782,2005-11-20,true,1310430736.6,5:22:43.27 +1917672332,83588901415583088,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.782,2003-09-20,false,132514049.735,3:55:50.27 +1933120256,51795844802803792,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.782,2000-02-04,false,534042504.832,18:57:28.39 +152477480,79062260226490096,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.782,1975-08-06,true,184740211.696,11:35:22.14 +772766666,15474504164415294,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.782,1969-11-23,true,1335964354.73,21:44:34.21 +-27192503,74405245232338224,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.783,1987-07-18,false,1217087565.09,2:2:24.3 +1731767828,87724475048404672,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.783,1989-01-16,true,1118995897.31,8:29:38.35 +-1174404474,31541895260663892,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.783,1959-04-22,true,551708734.335,20:32:45.11 +-59510352,57826141940581888,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.783,1971-07-14,true,10030017.3806,17:1:25.3 +1252805968,14828181422228030,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.783,1999-06-04,true,1328949176.99,14:29:27.36 +-1970852398,69894661029685024,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.783,1994-02-08,true,863190465.106,8:6:13.55 +-1070215847,82047355242076336,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.783,1974-08-27,false,918656644.117,7:43:22.31 +956708923,69364273189224416,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.784,1955-09-18,false,876053304.85,3:32:31.29 +1711754839,5642035719269878,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.784,1975-09-06,true,355731513.323,18:39:19.19 +1936138324,64822621540055800,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.784,1994-05-06,true,653557701.579,2:21:36.25 +1457394411,7008800354690294,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.784,1971-11-11,true,141263308.031,3:41:19.50 +1313828405,47495598805659616,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.784,1995-06-20,true,785759111.849,14:42:40.5 +1751093605,58639206636245488,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.784,2010-07-03,true,1062031907.69,18:46:31.55 +-322360424,73565630058135360,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.784,1981-06-13,true,815646971.069,18:42:51.24 +535166588,53115369166751560,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.784,1987-02-05,false,1022750307.05,18:11:39.27 +1492266737,30629738705569364,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.785,1981-03-25,true,412169590.816,3:3:51.37 +1599326435,31005061774933924,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.785,2000-02-26,true,248347849.909,16:14:24.46 +-1538448997,30476640418562920,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.785,1989-02-23,false,801500349.273,9:6:40.40 +799125859,36955406734876848,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.785,1954-04-05,true,20525643.0378,10:57:32.52 +1403257548,40791411282460368,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.785,2008-05-15,false,197732477.919,11:58:18.49 +-249892999,59394704790386816,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.785,1968-01-12,true,1188076598.68,5:25:42.40 +1047502929,21758093098704764,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.785,1981-04-06,false,508137353.382,15:31:8.27 +-1848761854,8608177783219108,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.786,1959-08-08,false,547651368.741,1:43:44.49 +-1914298185,14590837812707800,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.786,1987-04-15,false,325947212.322,5:2:39.11 +-1526732448,30631189156250768,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.786,1981-09-01,true,16780647.1831,5:55:29.26 +1326770562,18111708567101664,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.786,1980-02-24,false,923939770.296,5:1:53.6 +-303155102,55096228913731248,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.786,1972-04-18,false,1370313154.99,15:49:12.54 +522464839,22203547771548108,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.786,1975-02-22,false,920868803.558,14:46:57.55 +-1972933248,50878962181648464,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.786,1996-02-18,true,1080610291.31,13:31:14.57 +-1207276940,30173740511898432,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.787,1994-10-09,true,1285460821.99,16:43:19.36 +4336906,74627009785955520,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.787,1961-01-02,false,1316176675.03,17:44:15.50 +619465748,57287972323078224,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.787,1951-05-26,false,826634942.166,17:16:48.21 +-1284621646,17999403114350818,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.787,1964-02-16,true,1258815974.92,8:47:10.21 +-1161412154,80753534243955248,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.787,2006-05-19,true,637492680.103,11:47:18.30 +1031619579,10216249620704246,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.787,1967-08-05,false,59837093.3831,15:15:17.33 +81816199,42505941234318728,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.787,1975-05-19,false,207619210.94,12:34:17.21 +1735504321,865182989680711,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.788,1989-11-26,true,683236560.772,3:35:57.11 +-368338585,37547040989598448,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.788,1958-08-03,false,1059924966.21,14:30:12.45 +698673293,9702838945253458,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.788,2006-07-02,false,920206199.527,5:13:33.19 +-1623882774,80143476736919024,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.788,1968-02-01,false,236311371.009,7:58:44.41 +1928378120,1912107250991554,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.788,1985-03-13,true,1162145064.13,1:25:23.56 +-1418753503,88876468399360080,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.788,1971-10-22,false,1120653181.84,9:38:24.4 +1752384037,75498058737088368,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.788,2011-06-18,true,1098941593.18,10:48:13.9 +-900690046,62398673612731016,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.789,1954-02-07,true,223658582.913,19:6:53.21 +667012434,79205144438269824,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.789,2002-09-15,false,1216439883.54,14:48:9.32 +1937080377,17974395244881480,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.789,1960-05-19,true,1066803631.81,18:14:36.21 +-1118368658,45365535230509416,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.789,1952-03-23,false,742583160.301,19:46:4.51 +1387879688,48268049500429048,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.789,1988-08-16,true,33183751.7868,9:43:49.4 +1270082537,86215719998152192,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.789,2006-10-05,false,488215021.078,5:33:10.51 +197451167,73109124695165824,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.789,1952-02-18,true,297275225.539,7:11:27.30 +-1209958991,42453989041323160,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.790,1996-07-10,true,410387026.198,17:3:7.28 +1421696400,27123515853564980,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.790,1984-06-26,false,997223126.106,15:1:48.27 +1874935210,27968236893365188,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.790,2011-06-14,true,202297678.894,5:13:19.11 +442154313,77650295497768400,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.790,1980-03-10,true,945965686.467,10:42:46.44 +-631334061,6458066531818353,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.790,1996-05-26,true,1429523202.03,1:49:33.52 +33369068,84562416625616240,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.790,1983-06-19,true,175581209.403,15:10:36.47 +-1253045871,41492166902083680,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.790,1960-03-19,true,752744694.132,15:28:29.37 +-2027374127,57137633663484160,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.791,2004-11-24,false,240587550.385,16:50:19.3 +-90135085,57203856949112544,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.791,2008-05-25,false,411545905.823,1:3:51.17 +1927914923,38382550565162232,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.791,1951-04-24,true,461795165.73,18:47:35.20 +491857785,10744244566217340,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.791,1979-02-15,false,891853893.359,18:25:22.5 +2109552142,85105348983824400,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.791,1987-10-06,true,108526692.897,3:43:25.12 +1972945428,48027852912150280,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.791,1960-04-21,false,355173150.439,22:47:58.17 +365596424,26286040837600808,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.791,1974-02-20,false,104534338.343,10:31:43.47 +-596027096,45283423567462984,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.792,2001-09-23,false,566459687.365,13:22:31.21 +580968361,13781932105291828,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.792,2013-01-26,false,997736601.426,3:50:46.4 +-163462242,31540167462526956,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.792,1983-05-08,false,1124767459.02,13:6:33.42 +1957504906,29629982074128796,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.792,1995-04-06,true,437819429.535,8:13:51.23 +642067557,74656237214442288,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.792,1967-04-11,true,362273706.151,20:54:11.42 +1109280813,33813127729785160,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.792,2008-01-09,false,958652403.206,2:38:41.57 +308367112,34158211000165212,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.792,1960-10-08,true,123923124.516,16:10:29.50 +-1134167576,88332509937428288,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.793,1970-07-03,true,618303115.352,8:28:44.4 +825791011,65084641379058176,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.793,1955-09-27,true,956355314.612,3:7:15.18 +-1099609162,6291600098388101,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.793,1986-07-19,false,737016858.021,7:28:45.8 +120998060,78064477391984496,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.793,2003-04-06,true,142564699.497,22:48:44.14 +-1780066329,63994089988263664,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.793,1998-11-20,false,1126414453.14,6:16:3.48 +-1946394090,54299384318176592,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.793,1985-06-27,false,958401574.731,10:14:6.40 +-179349696,62638759037041144,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.793,1967-06-11,false,306223069.266,13:24:4.47 +533222700,80305490211839760,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.794,1959-03-24,false,67655734.9099,15:29:19.5 +1969240935,85884592633018720,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.794,2010-10-22,true,671604967.171,9:58:9.38 +257764258,73304640635929952,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.794,1964-05-18,true,227193324.778,12:12:14.9 +-1314061240,18505488192047812,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.794,2008-04-21,false,937285625.231,11:39:30.55 +573100400,71705398036211768,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.794,1999-06-03,true,6286823.94712,1:51:16.34 +1322826919,52407525500382448,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.794,1978-11-10,false,92154019.4279,13:14:21.56 +-1182035898,36937675514398904,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.794,1963-09-26,true,1161188515.15,14:13:37.23 +-778583244,44615547382381200,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.795,1997-11-14,true,1016679928.51,22:9:29.43 +946636171,72615441506225120,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.795,1990-07-20,true,142903398.613,17:9:26.58 +1352019644,26813128916493424,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.795,2002-01-04,false,805621318.985,16:47:51.48 +-884941708,34910904876574804,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.795,1993-02-25,true,355393654.54,6:41:23.49 +1373937308,48246589719125904,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.795,1985-08-03,false,1075809437.91,13:9:51.31 +-1479673884,50822090854611208,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.795,1955-03-22,false,141606284.81,5:35:29.3 +1834278028,47981877845191272,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.795,1997-08-11,true,405848312.755,9:11:7.52 +-226647870,48140844810815712,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.795,1961-10-10,false,560377205.734,9:25:21.28 +92225110,14601316676000256,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.796,1962-09-21,false,1081731097.91,3:19:51.22 +-1792405617,52628527961738496,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.796,1967-10-13,false,1011557107.13,2:43:1.21 +599620745,2343763005184573,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.796,1981-06-06,true,1029137655.51,17:52:39.15 +-1735327081,80868149952930864,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.796,2002-01-23,true,590556672.563,12:37:54.39 +856346175,35161365783177860,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.796,1951-05-22,false,768781219.776,1:12:13.54 +-1978157988,85512235406872768,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.796,1954-08-04,false,1260748300.46,14:47:25.17 +-1333746513,91229230953315968,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.796,2001-05-17,true,706899100.937,5:35:9.58 +-1822244567,40439885201957688,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.797,1978-01-24,false,653077727.707,7:30:54.1 +2076808454,34296601349351232,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.797,1987-06-21,false,845833224.99,14:24:43.23 +-1071006915,60115587055514688,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.797,1960-09-22,true,873093679.447,15:36:57.45 +-1722049732,57385787615775936,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.797,1952-07-27,false,28663639.7042,17:18:57.13 +-173059733,79091533652020384,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.797,1967-03-11,true,910038490.76,17:32:56.25 +1540534954,34981562369689304,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.797,2012-05-23,true,278827978.96,3:35:34.18 +-2147282778,41071726206448480,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.797,1991-01-20,true,85276951.6086,21:48:28.55 +-2085225880,18333362343832168,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.798,1951-03-16,true,1039239562.18,16:56:2.2 +-1008786468,7242192051520492,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.798,2010-05-23,false,183941780.314,21:41:51.23 +-206197913,12109227811688642,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.798,1962-02-24,false,1279006380.67,10:36:17.45 +196072404,90900977799001168,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.798,1985-05-26,true,1310407822.4,15:58:9.56 +-857832924,29703769056429548,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.798,2006-05-24,false,754269701.158,6:1:33.20 +-751380262,37282638633269168,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.798,2008-03-03,false,942445996.504,13:1:44.34 +-1414172825,2259161663220183,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.799,2001-06-07,true,178522242.386,21:31:49.11 +1455406304,86823966051004208,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.799,1950-03-06,true,709587898.181,4:56:48.40 +-1208228530,22803383521522104,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.799,1963-03-10,true,91820198.9227,17:38:4.50 +-2057855287,85558539797330416,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.799,1963-01-24,true,1408979448.79,15:37:5.7 +-988079856,13867199800184966,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.799,1964-07-01,true,489006906.409,10:26:10.27 +2033693683,51964901197114464,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.799,1962-06-03,false,931504928.587,1:43:46.51 +-1162672356,63316450658034816,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.799,1994-03-25,true,1004988469.64,20:18:27.11 +-1405309703,55815199950204928,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.800,1980-07-16,true,1329802667.91,2:7:58.23 +784462615,33971795647011360,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.800,1960-10-19,false,1229935610.25,12:41:15.49 +-1317440793,7874867708999,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.800,2007-09-15,true,946882832.865,18:25:36.52 +-209008447,69229326985108880,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.800,1954-05-04,false,751759871.239,13:6:12.13 +1979953234,48005431993603984,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.800,1988-08-23,false,796258495.874,17:35:1.14 +-1676648607,6045279834487040,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.800,2001-05-01,true,1339481162.73,8:3:2.47 +1632371528,41774655569684920,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.800,1997-07-07,false,723821694.556,11:18:37.27 +-1411640906,76954767096240704,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.800,1999-10-11,true,1106129637.93,11:5:46.2 +620988755,73430036228039920,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.801,2003-01-07,false,1426626229.6,12:53:7.35 +-933914567,55477897656196160,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.801,2011-09-14,true,687556733.354,1:48:41.43 +-1416794569,24723363411197020,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.801,1952-09-25,false,550892728.308,21:30:21.41 +1468855573,48557943926797784,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.801,1977-04-07,true,450605608.09,21:38:3.58 +-1298100887,84714473460969792,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.801,1989-05-22,true,1119713023.78,8:24:37.2 +-1247971304,42607529915825400,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.801,1969-02-10,false,350959437.67,15:27:56.54 +1118317240,86220445779902096,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.801,1996-07-19,true,609705834.944,7:50:34.10 +-211128879,72302501605669888,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.802,2006-01-07,true,1301177609.74,6:15:18.44 +-1473906729,87655978960360944,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.802,1987-08-07,false,1060822654.73,10:37:22.1 +-112077875,54397803329780656,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.802,2003-03-02,true,193507846.004,5:11:23.48 +-150805766,30533168542465608,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.802,1962-05-23,true,766473994.947,20:50:54.41 +-89731207,29239200366514976,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.802,1973-10-19,false,970590035.558,19:55:8.23 +2141952968,84801430910272192,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.802,1960-11-15,false,891493637.841,3:42:32.38 +-207286959,63626020113783008,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.802,1986-08-19,false,1372544284.87,5:46:56.58 +-120638765,11063745565164084,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.802,1980-08-09,true,178963160.055,18:56:21.1 +-860550329,56944606256766592,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.803,1988-06-15,false,329486061.731,11:21:32.44 +1272089158,68465625934726944,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.803,1988-01-08,false,137540325.951,5:48:6.42 +-578167285,70892173530625080,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.803,1968-05-03,false,827053047.639,20:41:3.7 +-31212860,60903930937556208,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.803,1952-05-03,false,198340989.712,17:43:56.45 +-849054763,64503894514732360,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.803,1973-07-21,true,971200465.132,9:11:25.43 +449088639,62682423424915480,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.803,1980-07-04,false,40313113.4478,18:39:27.24 +-1391662309,77990596056005360,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.803,1962-08-09,false,645805240.95,15:37:9.24 +1980742827,82764428165369424,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.804,1970-06-09,false,391058153.119,2:31:54.32 +-1506279946,81336368267951008,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.804,1977-04-15,false,983086814.008,16:40:9.28 +1277648327,83031229431960784,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.804,1950-07-07,true,332846415.869,16:33:54.15 +1088656111,47541025892605464,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.804,1953-06-13,false,333356990.749,11:8:20.32 +-1283411791,76476644281595456,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.804,1969-05-27,true,1269803128.55,21:16:22.22 +-1722776353,76522649454006880,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.804,1970-11-08,true,1001217291.23,16:51:44.40 +-487208478,20113141298102508,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.804,1972-10-05,false,1292421805.53,14:9:46.25 +-1404695186,87764591079662064,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.805,1993-08-01,false,1099264303.7,20:1:29.57 +1961882183,47140098580926728,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.805,1968-07-09,false,344174786.055,19:24:43.36 +-1552515709,62711532568867472,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.805,1966-02-12,false,817526131.005,6:21:52.20 +-659451302,87067566634269600,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.805,2001-03-26,false,266990717.793,17:49:45.46 +-627295324,75059336780617984,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.805,1971-11-16,false,515202139.83,5:41:4.7 +-1340374074,83997505183766464,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.805,1954-04-22,false,679267899.379,12:57:34.46 +-1452011444,22432142691672832,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.805,1968-05-25,true,1263201755.92,3:54:53.39 +1411716834,73586432303359504,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.806,1962-05-24,true,1389447202.07,13:51:57.55 +744930217,84515591338446464,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.806,1959-11-09,false,1330977660.66,1:50:12.56 +-120892846,90136953214397664,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.806,1991-05-10,false,850965313.507,22:28:21.18 +794171138,15699515309808732,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.806,1991-10-27,true,1353410794.01,8:50:1.27 +-1574597969,13010433236026266,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.806,1980-07-16,true,1190307260.21,11:7:50.9 +-1706108052,61385354944377984,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.806,2006-08-03,true,945305185.163,7:6:35.17 +109836992,59103449355830824,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.806,1999-08-03,false,455402989.259,6:22:47.51 +-1696283352,87592497625405376,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.807,2001-11-10,false,970193758.097,8:13:39.53 +182457148,18804745495955876,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.807,2009-01-22,false,302509000.01,1:11:13.31 +1143092469,29275194143011716,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.807,1968-09-05,true,846554820.21,20:55:14.1 +823075255,84164537061018640,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.807,1968-06-26,true,30542314.6559,6:45:5.46 +-624037344,32072260032496568,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.807,1959-07-12,false,479773620.863,7:57:37.26 +-1663575811,44178674776738840,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.807,1988-08-04,true,1183048136.93,22:20:11.37 +1959344156,88428436679676112,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.807,1966-03-24,true,1171320482.68,13:55:9.36 +1337596696,10017746921423216,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.808,2003-05-09,true,1322432359.39,21:52:7.26 +915984729,50122276206766560,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.808,2004-03-12,false,1195666415.83,11:43:31.1 +435808134,73094926508328704,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.808,1985-02-04,false,873110871.13,19:9:18.13 +-1505704420,63336270521526712,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.808,1958-01-09,false,1191574251.42,12:22:25.15 +79944608,40354836592533400,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.808,2007-02-27,true,1079939518.23,20:15:22.4 +-1705538687,60454845662709376,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.808,1996-11-03,false,576861498.144,21:30:9.22 +-1101860289,33093574725110336,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.808,1969-02-05,false,1113491031.58,4:30:39.35 +-833043704,72716762021985536,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.809,1957-05-01,false,145585803.355,15:54:16.53 +1768298080,14971197838271878,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.809,1951-09-12,true,163872774.495,7:55:36.55 +1085358642,49039064999378568,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.809,2003-11-05,true,1335519904.34,19:48:2.13 +-1708480728,6226553587018168,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.809,1991-05-21,false,655096287.89,8:57:32.4 +1733331469,2466508617753108,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.809,1974-07-19,false,1420954470.53,6:41:48.31 +80319376,72115138462685504,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.809,1967-06-05,true,477229538.568,1:36:12.9 +-1636202535,14339080998355160,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.809,1991-01-07,true,298009493.073,5:20:13.43 +-2083987404,16046346893276386,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.810,1954-05-06,true,321096270.667,4:27:45.32 +-2131101676,63378948779531144,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.810,1964-05-22,false,1056146417.85,7:5:57.51 +-1747852334,86146053700827200,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.810,1976-05-26,false,1051471728.78,17:42:44.4 +-25341089,33734350931543964,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.810,2001-04-04,true,116164919.67,4:33:51.49 +-1379702150,3058331977209364,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.810,1999-10-14,false,577701546.639,21:24:48.3 +-134273811,39298278293955256,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.810,1957-08-17,false,502451957.736,12:5:53.7 +-367984701,25130314371149476,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.810,1998-05-15,false,764142214.328,7:41:47.11 +1483829298,75632092825130624,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.810,1997-10-11,true,1283957337.97,11:18:47.30 +1071062324,44492252398910968,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.811,1992-02-13,true,189274074.575,16:42:32.24 +-1955331657,22893770654007532,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.811,2004-06-10,true,338794495.303,8:4:16.23 +2107979734,62671563387160936,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.811,1969-09-22,true,1198820782.59,8:28:52.5 +-1989118171,32372039130780572,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.811,1951-11-25,false,838443211.291,5:27:23.41 +-931930737,57708251929905104,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.811,1969-11-26,false,931767978.511,18:58:37.6 +-771366593,45780463035331216,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.811,1987-02-01,true,465080151.748,20:4:44.14 +1561047106,22076871521801676,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.811,1959-02-03,true,867295963.067,1:3:1.7 +1252537526,86927814298296384,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.812,2000-10-27,false,1138916732.7,17:1:50.31 +-868082784,12646002281026632,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.812,1951-06-05,true,326742613.171,5:35:6.42 +-126974574,42704405166154632,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.812,1969-10-10,false,199070502.235,5:21:38.56 +-397130277,57916022532412480,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.812,1987-07-03,true,1107364644.48,13:22:33.33 +-1188510306,37065858327788544,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.812,1975-01-19,false,1258322057.75,14:34:46.16 +-2021344862,5306367701051679,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.812,1968-04-13,false,402078054.247,5:53:24.39 +-1198927391,59102272466651432,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.812,2007-06-15,true,424929782.544,6:50:15.2 +810664626,62767650733331384,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.813,1977-05-20,false,1105423583.71,6:50:12.18 +-2036594628,30024200414973912,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.813,1991-01-02,false,221218633.76,16:44:41.39 +-1132954490,11757646599716854,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.813,1980-03-03,true,439717967.941,2:43:35.16 +-1290853532,43544001100559792,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.813,1973-01-22,false,626651094.556,16:48:24.17 +-1192115466,79392594051456576,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.813,1962-08-04,false,513861051.818,20:20:32.30 +1371856834,13518308495783384,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.813,1964-08-07,false,199353347.769,15:27:45.11 +1951036196,91169956000755440,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.813,2007-06-03,true,1166865078.93,21:43:21.1 +-272826302,8140719744311399,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.814,1987-11-15,false,228105597.826,6:3:31.46 +-1118534935,14569603182725396,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.814,2006-11-10,false,879899678.081,19:30:32.12 +1956978070,87907710261299568,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.814,1955-04-25,false,209158621.768,1:40:8.29 +-1152882069,64257602287053704,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.814,2012-06-12,true,1277131689.13,19:53:20.5 +-1584874949,60117222060352728,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.814,1955-09-08,true,686208761.407,21:45:13.24 +357728418,50819944526506432,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.814,1956-09-08,false,291792536.376,4:42:19.7 +902842216,35904359570179052,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.814,1990-11-08,true,990751230.769,13:16:16.35 +-240426858,12630430845880924,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.814,2007-08-25,false,1076554522.43,5:13:21.2 +-1159966937,83267223655436880,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.815,1972-04-22,false,781286295.654,1:31:46.28 +1925118847,3917078965304443,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.815,1962-05-01,true,989733940.937,12:10:57.20 +-1068944975,20774909808724880,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.815,1966-02-04,false,1228014816.51,21:47:6.48 +-580947178,4866326821663898,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.815,1956-08-25,false,1415050959.95,6:44:2.48 +326393788,36243432487945320,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.815,1986-07-23,false,536621289.196,5:33:57.12 +612267368,44360091407613136,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.815,1969-11-02,true,666583972.95,14:47:34.56 +1563157315,37504594750296944,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.815,1955-11-14,true,223116602.465,21:38:12.54 +16168892,18031742120208652,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.816,1995-05-03,true,1064813632.83,5:12:49.38 +772493225,47864038953383904,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.816,1952-03-18,true,1381315070.27,10:25:18.13 +1487183570,6208716410980393,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.816,1992-11-01,true,1185350618.86,3:55:53.39 +2078638433,34437009988448204,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.816,1998-10-01,true,794968964.135,3:35:17.37 +-1392450585,64014244435049648,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.816,2001-08-23,true,1385895561.06,12:17:55.35 +1289650582,53263500569229888,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.816,1955-09-10,true,79148590.2372,15:45:22.36 +-2102193239,73963742991835184,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.816,1979-11-19,false,57215290.8847,9:55:7.46 +1596191284,70596082272071560,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.816,1983-04-12,false,1372383231.05,14:1:45.48 +-630611905,63256935126637752,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.817,1970-06-19,false,112721547.718,14:33:46.27 +486114678,70146454393529000,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.817,1967-04-07,true,590987955.267,20:34:18.25 +-71888833,29369659659154096,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.817,1997-03-25,true,1135623387.19,18:11:55.8 +811877079,43745504794229560,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.817,2004-02-15,false,1057589534.68,3:42:16.21 +1502152319,7014988311369083,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.817,2009-04-26,true,1191827033.97,8:5:25.28 +935334081,65199030562993344,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.817,1969-08-04,true,1394205068.69,4:57:46.26 +-219560415,59441961475847304,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.817,1978-10-01,true,1153932874.4,16:17:25.17 +989205304,33261166481930772,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.818,1984-05-01,true,1364580059.37,16:27:40.58 +-1911776921,56856659023978344,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.818,2009-11-15,false,973747520.371,13:14:40.53 +1866196111,88157055970592368,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.818,2005-01-25,false,392923113.409,21:51:39.34 +334364521,56794629820780720,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.818,1988-05-09,false,414732349.778,5:37:42.7 +2095807313,76719530893982080,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.818,1950-08-20,true,587733562.237,12:55:11.47 +922881735,72185745614391344,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.818,1980-07-22,false,1352857233.16,21:48:38.31 +-784052382,10336676307316020,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.819,1979-03-16,false,841595572.538,8:39:6.37 +-1877344185,54940504055139352,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.819,1951-08-01,false,1246973063.99,7:33:46.21 +350136416,61858595398102624,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.819,2010-10-11,true,1364617905.76,8:54:57.18 +-336455057,16008356911299574,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.819,2002-05-21,false,964237145.25,20:28:56.38 +-1486338721,42824542154393152,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.819,1984-02-11,true,1263179141.43,20:33:38.48 +-458620182,89827597661495488,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.819,1995-08-01,false,140577704.366,14:54:3.32 +-1215512347,25802429831127656,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.819,1969-08-19,true,383889616.329,7:7:8.1 +-1257298084,80496756509505632,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.820,1957-06-03,false,1314045590.59,7:24:41.31 +342201652,53269303797531320,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.820,1974-06-11,true,795087764.611,1:6:38.36 +1721783553,70993838226074512,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.820,1971-05-10,true,533854873.496,12:26:7.1 +1670598113,71666341129686600,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.820,1961-09-14,true,352310323.274,20:4:51.26 +-1707720285,81104947237995344,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.820,1971-04-26,false,1059166294.88,12:43:2.4 +312839556,34472170853430016,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.820,1968-09-08,true,2911395.39788,14:57:32.7 +-198645182,31120614712474728,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.820,1972-02-04,true,1028603794.06,13:8:9.46 +-402349473,37224745665765936,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.821,1956-07-27,false,479326778.099,12:54:26.44 +-412820848,2438436230933862,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.821,1951-05-02,true,323440715.011,20:45:48.20 +1245436564,25705925162103400,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.821,1978-10-26,true,1062996979.77,1:21:29.39 +-778424069,17955079332767530,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.821,1962-04-17,true,1086757770.23,15:10:20.2 +1570344727,32208929622770996,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.821,1984-07-13,true,1256262572.84,8:35:2.35 +-1703879772,75490005203045168,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.821,2003-07-24,false,593087091.374,21:19:31.26 +-827576993,39538083190087464,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.821,1964-03-01,false,522385197.845,2:45:18.43 +1606477199,34924598214089444,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.822,1954-01-22,false,62991229.6226,21:41:17.16 +1533293147,58382860624317288,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.822,1986-09-09,true,675443779.074,19:9:55.23 +1702384986,51244507588129568,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.822,1995-02-05,false,1073981512.47,12:2:46.48 +-445994610,58564530514068520,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.822,2006-06-14,false,951138805.408,15:1:50.44 +-307179481,47916789104570832,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.822,1975-06-26,true,158089760.095,15:25:27.43 +-387604251,41594164323400808,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.822,1991-09-07,false,824502943.211,15:16:43.47 +-1944382784,63273414005066712,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.822,2008-07-10,true,200576729.28,10:50:53.2 +176043413,51044644452721264,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.823,1961-05-03,true,1022375955.21,6:33:20.25 +1557167107,71173979637101920,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.823,1951-05-24,false,911105912.476,13:48:56.53 +1636324099,36185292224042448,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.823,1973-10-25,false,1308892228.91,12:13:48.3 +-969385338,86268261733252704,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.823,1982-04-19,true,764707251.519,1:39:14.13 +-268871934,20395134810404352,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.823,1994-07-20,false,736919378.881,16:32:3.49 +745540099,58735347884770840,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.823,1963-08-13,true,1382587027.17,18:41:50.56 +-1582055023,84394305638508128,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.824,1976-08-09,true,176606363.979,2:56:3.42 +-1527876492,2173289610746316,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.824,1965-03-13,false,684912524.374,9:21:19.11 +-599270830,51804678866847192,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.824,1994-04-05,false,644771539.393,6:29:32.36 +-1689102822,71506513393495760,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.824,1972-10-22,false,104088035.146,16:11:35.10 +-995788545,37786723560149640,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.825,1955-05-17,false,1110781923.42,21:20:16.55 +-2079736618,27313747965662464,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.825,1960-11-07,false,60266484.9455,22:37:33.46 +1320056643,72147606741124208,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.825,1979-05-23,true,454615227.255,6:22:2.6 +-574586452,39410913835019560,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.825,1966-10-01,false,732110296.617,8:7:37.55 +-1749608661,39301780049212560,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.825,1965-07-09,false,1388652002.06,9:40:16.40 +991278715,1213884129990830,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.825,1984-07-17,false,1338119311.31,16:41:39.24 +-1879487361,27200727541424352,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.825,1982-03-17,false,796409264.956,6:55:58.56 +-104712292,73846353440629648,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.825,1999-02-14,false,1199790778.65,1:52:28.48 +536017333,63462922466710184,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.826,1988-01-11,true,221057497.881,14:12:13.1 +1425527071,37810297590556608,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.826,1985-10-20,false,221500935.996,12:58:1.15 +1865928557,1311715821568419,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.826,1991-11-14,false,482239979.916,20:14:53.3 +173962929,30864560674618440,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.826,1976-02-01,false,916384268.244,22:33:43.36 +-954790771,25612263798367744,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.826,1953-03-21,false,835489296.938,7:35:2.39 +1850975163,25456881420899676,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.826,1987-03-23,false,1392562684.12,21:31:53.38 +-1304636755,59828021674054952,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.826,2005-06-03,false,1311887196.29,1:7:28.1 +-8663844,47586219466478104,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.827,1992-05-01,true,375758271.717,18:18:42.4 +-622130286,34863051463690568,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.827,1989-06-25,false,425233536.426,16:48:46.43 +1746086862,77208775570441552,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.827,2008-05-14,true,962182127.642,6:13:19.58 +1261017409,47943122513553944,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.827,1972-09-24,false,806515463.632,10:48:31.9 +-1990323816,45339868313908920,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.827,1994-09-23,true,509084642.513,9:15:28.6 +-1605216800,23130918521549384,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.827,2003-07-02,false,1430094415.15,20:37:46.5 +840883499,87787206784817536,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.827,1984-04-17,false,1137611450.08,15:32:45.36 +-1736452084,79772808184483552,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.827,1961-06-25,true,863247517.493,5:13:27.30 +-1447488361,26660353791524568,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.828,1965-08-22,false,355258816.715,18:24:16.48 +197841331,66503830823209480,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.828,2008-07-23,false,191238399.804,19:32:57.19 +394022442,82708903009147872,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.828,1962-04-06,true,1246002450.59,5:37:48.5 +197800209,13725505791346668,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.828,2007-08-27,true,1013039516.75,15:26:20.23 +275235625,11650807564644608,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.828,1998-07-11,true,709105963.053,1:58:19.41 +1748027347,62962384222898304,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.828,1969-08-09,false,1334872160.63,13:39:23.54 +-1217450083,90427680660945056,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.829,1951-11-24,false,390067136.978,14:35:23.26 +-657179448,24498192362004336,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.829,1968-07-05,true,8579905.19403,17:35:23.47 +1670360344,69862365872162944,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.829,1974-03-23,true,674754227.049,17:10:11.46 +-1193294076,14784127463440866,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.829,2008-02-01,true,714774927.577,9:13:13.32 +1993586488,67903577374359512,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.829,1978-11-05,true,974627308.969,22:5:40.20 +667930476,68481466217052712,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.829,1960-08-22,true,444120834.374,19:9:17.20 +-298188887,67495340308590912,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.829,1986-07-22,false,1119013767.05,22:36:10.58 +-234508530,73462286232041920,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.830,1983-11-23,true,624586517.444,14:22:54.11 +753796747,223946121159024,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.830,1984-07-05,false,1424152211.75,18:33:23.3 +2058999982,86843953283434928,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.830,1994-10-03,false,1154872839.07,1:49:47.16 +26469064,29256437982596352,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.830,1976-02-20,false,175317679.371,14:32:31.6 +947485729,46047986351436312,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.830,1962-01-24,false,567295294.566,7:58:32.27 +-2079542265,60641942942889352,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.830,1999-09-02,false,446262219.744,16:37:53.44 +563105111,34807478770912320,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.830,1967-08-20,true,522544522.034,5:20:25.36 +217516375,16008604573623214,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.831,1992-09-12,false,753391537.686,2:40:21.47 +-677248392,62077283798258784,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.831,1996-06-07,false,250140956.152,15:38:24.15 +71951550,22940672353058940,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.831,2000-04-06,false,236241568.68,22:43:30.48 +747231876,7155405525724682,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.831,1989-09-09,false,580287468.139,18:31:34.28 +-431157378,9332183104061184,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.831,1995-06-13,false,97975513.1066,10:37:38.4 +-1343410867,64882832716395592,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.831,2005-04-06,false,1098956506.62,3:47:58.43 +-798667741,8028326733767127,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.831,1955-07-18,true,489278189.514,15:16:8.33 +-1128553072,38981377044889512,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.832,2013-10-14,true,1035906893.0,13:30:16.45 +1608512642,46610129706096128,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.832,2008-10-12,false,1009170585.24,17:14:4.17 +-980708690,5882011352411484,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.832,2005-05-12,true,306813369.338,3:21:7.14 +940500242,63310133091867584,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.832,1972-05-24,true,1254770856.66,9:21:21.26 +-363833152,8511026537224182,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.832,1965-10-13,true,164520613.238,17:49:52.54 +1671509770,6883777398846546,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.832,1951-01-05,false,1119153682.59,18:53:39.27 +-893834900,30135815677079316,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.832,2000-07-20,false,396477801.461,21:43:6.31 +580403463,38097690425567656,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.833,1981-05-19,true,599137556.231,20:47:4.30 +-1397788545,44065223109204000,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.833,2003-11-19,false,1255120285.2,5:40:36.50 +-253625610,75692689867000080,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.833,1965-08-09,true,1226766617.27,1:26:37.43 +1321520423,25462751267291568,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.833,1975-05-19,false,381137494.677,21:8:42.54 +1488434898,85029999923113456,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.833,1957-10-09,false,927864386.615,19:7:45.1 +-762374386,384831565449369,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.833,2005-08-17,true,1393957674.08,5:2:36.36 +-1864028874,26681057351850128,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.833,1970-06-27,true,503733184.406,20:4:55.35 +1328988626,33173906719554192,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.834,2003-03-11,false,1318871580.86,11:10:7.4 +1596541397,89122902213611712,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.834,2001-10-04,true,486194389.256,4:25:8.5 +1166929833,90716639067232304,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.834,1963-03-07,true,1260700239.6,13:13:11.34 +-367065099,86883121134400096,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.834,1956-01-03,true,373407872.465,3:35:9.31 +-822860573,26019164437585060,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.834,1982-09-20,true,754437844.287,10:14:24.40 +2126670146,6609276601893601,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.834,1975-10-07,false,465743340.772,21:31:28.33 +294903793,14705159662183260,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.834,1986-02-09,false,1236544322.47,15:21:37.26 +-1128465725,46771694168041336,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.835,1986-10-27,false,388116082.233,15:41:41.50 +-74095438,35360616259264400,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.835,1951-01-10,true,427231712.33,16:46:57.6 +188902756,52521189692089920,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.835,1997-08-16,true,273765129.898,16:32:13.24 +-1835606043,35454231875329444,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.835,1959-11-05,false,1301806460.08,21:38:14.14 +-1256460083,86879161632459744,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.835,1981-11-11,true,161034156.0,18:28:40.16 +-2118593780,36764437251430992,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.835,1977-09-11,false,1072808007.72,16:47:2.25 +-2008939779,20891819539929304,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.835,1967-03-27,true,1147654113.83,18:6:35.20 +842900821,55185898865145552,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.836,1998-06-10,false,914413608.733,15:25:16.26 +654520522,24936078726529412,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.836,1957-09-17,true,449058345.358,8:16:9.39 +1127431459,24764296889713328,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.836,1978-02-23,true,331565706.466,7:46:28.36 +-1801240925,43368021442525392,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.836,1970-01-10,true,976307188.239,12:40:56.34 +-552067932,50952122553890264,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.836,1973-01-21,true,1144165499.73,19:35:4.19 +786644006,74305829963352432,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.836,1991-07-04,true,1342397594.11,17:33:18.25 +1905007605,19044742212714752,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.836,1951-11-14,false,1000511712.32,22:22:40.17 +-2115036224,88062436956053408,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.837,2012-03-22,true,1151153729.98,22:8:19.55 +1681032820,64836294759812520,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.837,2010-08-21,true,942925320.392,11:22:20.20 +-1300951605,78844461492003152,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.837,2008-07-04,false,779893203.76,15:52:23.20 +796963528,54187097099887144,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.837,1975-05-24,false,2830602.3356,1:11:42.46 +-987861677,49649863291618256,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.837,1976-01-04,true,157055037.337,17:38:10.35 +-158234848,68570559243328592,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.837,1952-04-14,false,43900491.1268,5:28:32.53 +-1641834866,2089618569649746,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.837,1958-08-07,false,191317601.8,20:54:35.26 +1439664984,63016630970521600,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.837,1992-05-15,false,1083360992.91,13:52:49.15 +-1126145654,74790435543620896,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.838,2012-11-03,true,135103215.768,7:26:15.11 +925720235,42555437735834208,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.838,1992-02-11,false,1115957248.11,19:12:55.15 +-2034082606,69915901553987952,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.838,1983-02-26,true,1042699524.88,10:49:26.40 +-616595309,83743746056369792,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.838,1984-10-02,false,1316670799.93,10:54:40.44 +1982682123,82663314796403216,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.838,1997-04-20,false,168385413.371,18:20:12.12 +819416754,16926178564649892,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.838,2005-01-12,true,232095203.133,8:46:35.46 +-402436566,7279168463777792,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.839,1992-11-13,true,1111964787.16,1:26:47.49 +-944304714,73363914979819232,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.839,1968-10-16,false,54104914.7556,8:36:19.13 +232323514,71195833054812408,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.839,2005-10-07,false,1360446444.76,4:42:14.47 +-994209174,80978848213568064,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.839,1985-01-02,false,1256455827.23,1:3:33.29 +-686099715,31852134346713568,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.839,1993-02-22,true,364756231.564,3:37:4.7 +-1461144466,10829941568362598,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.839,1996-09-07,true,672070753.542,13:39:55.57 +-834135713,89258387212783344,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.839,1950-09-16,true,1255275481.64,7:11:7.26 +2085007436,83714555849764880,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.840,1979-10-02,true,561544448.942,5:49:24.1 +1580999069,78332863865788576,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.840,1994-01-01,false,1095145421.86,10:55:5.56 +388324271,23388350351061044,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.840,1990-05-03,true,714498958.413,21:34:10.42 +-1925159181,37573621317963304,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.840,1971-10-13,true,623483153.672,5:51:22.19 +1011244146,37509014244665208,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.840,1990-11-05,true,119488690.6,14:15:18.8 +611548385,63000982224013136,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.840,1984-08-27,false,978234923.308,9:6:18.1 +-1469568125,83485066155972848,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.840,1998-09-04,false,16371665.0916,15:36:8.30 +-1235016969,36621708102085240,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.841,1957-01-04,false,750180843.89,9:44:51.48 +-1746182179,69240220000996776,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.841,2013-11-06,true,1086992564.01,3:27:23.41 +-669187720,11556785568136990,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.841,1954-04-13,true,936629027.868,22:56:32.41 +-1951557264,34850597412274700,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.841,2004-04-05,true,936391473.898,2:45:56.9 +809772616,28457477344184196,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.841,1974-10-15,false,654119274.144,13:52:51.19 +-1479572874,76195104664523232,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.841,2012-08-15,false,1025116584.2,12:6:23.18 +1544998547,7928896545245717,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.841,1973-09-16,false,859621081.234,20:49:46.58 +-1032390437,75718360576175552,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.842,2013-04-07,true,1416197238.31,8:29:12.16 +647255856,83728398179854272,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.842,2013-10-02,true,540159697.739,16:20:52.46 +1837354599,17797257799842520,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.842,1997-11-08,false,942826891.296,8:12:5.10 +-1676560726,38277004605610128,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.842,1991-05-12,true,1065934374.31,7:36:9.24 +384980140,91040828538764400,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.842,2006-11-27,false,136998889.693,8:22:19.35 +-1109112343,5353276858278318,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.842,1950-10-04,false,230928003.722,7:41:11.34 +1172590272,8151533493108613,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.842,2013-01-25,false,1052162191.66,5:32:13.48 +171360689,70649338758714736,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.843,1950-07-03,false,887314993.263,17:49:40.15 +1871579262,17724345905857198,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.843,2010-11-12,true,836759469.416,3:20:8.11 +-1477835300,19982277351997716,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.843,2007-11-21,true,1314387613.44,18:43:40.24 +149354748,36209714761008336,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.843,2010-04-14,false,129374851.795,22:16:10.40 +270177543,87867281625627760,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.843,1987-07-11,false,546181535.313,21:37:38.23 +-1459911440,53168469200617504,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.843,1965-06-07,false,1246224280.61,3:34:52.25 +-387517553,41972694296287080,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.843,2011-01-26,false,580629321.2,10:14:25.23 +-740091051,75725498011261920,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.843,1993-04-18,true,1083403452.42,1:4:33.32 +1246027236,13281118237975532,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.844,1989-06-16,true,336140708.028,11:16:30.19 +-590607658,56681833132470224,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.844,1969-09-14,true,22640372.1293,14:34:21.50 +1781658800,10242580042273772,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.844,1954-02-02,true,1100109268.13,22:28:17.40 +1755254786,35381688654784668,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.844,1967-05-01,false,818500841.101,1:8:53.40 +-1652848863,38389084570009960,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.844,2003-03-05,false,1080462977.34,15:54:33.31 +421858748,6982132269255291,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.844,1955-04-02,false,808380910.797,7:47:19.55 +1528418654,20518403454872104,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.844,2011-05-22,true,40473960.032,16:34:16.14 +840175501,45404314054715048,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.845,1999-09-14,true,403267585.728,2:21:25.47 +403826321,81938008704798400,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.845,1951-05-19,true,175040666.429,21:6:43.44 +-519004890,36297299566166208,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.845,2005-03-04,true,1329020227.33,19:42:37.49 +1767983303,92022246986507280,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.845,1951-01-04,false,1313663096.52,20:51:40.31 +945659059,797560899454945,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.845,1951-03-20,true,671672726.069,15:57:6.23 +-665332954,20104829101598372,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.845,2009-03-13,true,1209999482.2,13:11:16.51 +1420714748,68705978812513976,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.845,1992-09-05,true,1137199012.53,17:13:33.44 +-269793602,21146566223846840,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.846,2004-07-01,true,550744637.149,6:46:23.44 +266405925,86738114588096384,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.846,1971-05-23,true,625108466.6,13:54:10.15 +372529276,82097822622603344,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.846,2013-11-16,false,1050742104.44,21:19:28.37 +894677392,52143717443940520,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.846,1999-11-24,true,980870483.247,6:7:45.38 +-1938529851,56876622204488032,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.846,1963-01-25,false,643270997.67,13:26:38.42 +1495484808,34262355280117372,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.846,1976-07-16,false,1016360264.95,15:46:1.16 +-1691123437,67389514307697416,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.846,1990-04-13,true,74912319.2983,4:21:39.23 +1252758677,39134130751327592,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.847,1952-06-06,true,1233915180.43,21:27:1.19 +1623731290,88153719152330336,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.847,1998-05-04,false,1081075735.4,16:9:44.21 +1895010243,58302847802633600,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.847,1988-07-07,false,453758884.425,20:39:58.56 +-89515893,44522203072796880,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.847,2001-03-04,true,1044197351.31,14:28:35.44 +-1764238055,69175776074920272,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.847,1951-09-02,false,419825171.249,1:18:24.8 +735196068,62628796250546768,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.847,1958-02-05,false,236517606.69,22:40:49.14 +-1284997732,61223555963679120,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.847,1956-03-22,true,68213402.2125,3:5:14.8 +-2023065176,35600836872943052,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.848,1955-05-24,true,270987286.74,7:4:12.55 +1121527589,16117595386104002,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.848,2007-08-03,true,318900389.446,9:39:40.32 +-1530676373,1178807984424151,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.848,1996-08-14,false,1424276484.34,11:34:34.33 +-348477997,49345828569498848,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.848,2013-06-10,false,558447742.459,2:48:25.51 +1342972914,74956209071762256,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.848,1986-01-21,true,76378777.6043,15:43:54.19 +-1971466263,26966044519524692,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.848,1963-05-22,true,1060185626.64,3:39:26.6 +1348422854,14684413563258634,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.848,2002-04-14,false,1084346383.38,11:43:53.26 +1645928786,37779172979927280,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.849,1998-08-23,false,730870827.465,17:46:47.53 +17020476,39422845412640168,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.849,1974-04-05,true,137506447.031,12:39:56.8 +-337609772,69670194007230176,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.849,1997-02-06,true,484599378.542,7:30:52.9 +-14254154,13983217402291568,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.849,2003-09-20,true,1014278829.43,20:47:53.30 +-338396348,79667416981033760,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.849,1984-01-08,true,600111069.666,20:29:23.13 +761909867,45350761616992280,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.849,1965-02-14,true,1316675562.32,4:31:4.54 +266782700,79933895855721040,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.849,2006-04-05,true,1304091160.69,12:13:55.34 +81503130,83872084261212704,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.850,1982-09-12,true,453832089.073,16:28:5.46 +-957150916,36791852317534456,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.850,2009-10-15,false,970189265.512,6:2:7.47 +-1615817067,70077147045937728,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.850,1966-08-13,false,1030875280.09,2:5:54.17 +-1787417144,33421878121261260,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.850,1974-07-23,true,208930398.475,10:27:22.49 +351896951,35517047687668848,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.850,1983-03-17,true,764106435.295,19:57:44.31 +1512847011,64743840939680608,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.850,1972-01-11,true,954845378.335,2:22:31.7 +982764241,40953014630421256,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.850,1988-07-25,true,1256175586.72,19:16:3.37 +-1675514445,31297345610234900,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.851,1980-09-10,false,707100518.83,22:30:16.3 +332457734,58239265639644520,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.851,2010-05-14,false,23441439.4906,18:52:50.23 +578736740,55907859743816944,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.851,1970-04-15,false,598342102.964,15:43:6.22 +-561588193,30811246293399716,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.851,2013-08-08,true,1061019295.54,13:18:34.4 +409566959,357365197566597,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.851,1988-04-19,false,634016828.192,2:33:47.29 +1326794302,17122690797609830,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.851,1961-08-20,true,836668260.945,21:42:17.43 +-1605462605,40454691123087624,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.851,2011-04-12,true,524972789.149,10:26:14.33 +-591635739,70172285518960976,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.852,2001-06-09,true,13976758.1752,15:50:39.49 +-1846594683,53576977091906000,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.852,1972-10-21,true,230432944.363,13:54:14.14 +627361735,59407203422732352,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.852,1999-10-11,false,56236010.1054,4:30:46.18 +-1592997801,50304838492493152,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.852,1995-04-24,true,811307090.412,11:29:19.5 +-473741828,2756234259538360,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.852,1996-09-20,true,110220881.064,8:7:45.41 +-1618259552,64916148685014384,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.852,1975-04-27,true,1138578320.39,3:40:2.16 +-180571681,50987033754168088,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.852,1956-07-03,false,1395673255.1,16:5:19.12 +1175823298,69020134911516072,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.853,1955-06-19,false,705259795.233,5:5:13.7 +143855830,2252392243256822,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.853,1976-11-01,true,808554278.79,6:6:47.53 +1014367077,19082691451774116,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.853,1986-02-13,true,465427790.42,12:51:26.36 +420407861,40889340227845296,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.853,1956-05-19,true,1384285653.54,10:18:18.35 +1187282053,60403880925344688,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.853,1958-06-21,true,268836623.52,3:55:37.20 +-278073824,14315397304367534,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.853,1988-04-09,false,1186442005.06,4:28:43.25 +-225679884,73065722693304192,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.853,1966-01-19,false,403090457.488,10:5:55.19 +1208547529,55244898222403760,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.854,1974-09-02,true,1271702104.35,15:53:36.50 +-1418392799,54609284369154520,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.854,1962-02-12,true,942562288.003,16:10:52.34 +-929594822,52478576994016224,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.854,1988-11-05,true,76747511.1553,4:56:17.3 +-221677259,23412920923649536,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.854,1996-02-24,false,641613214.774,16:18:29.53 +-913850592,33779206072538316,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.854,1962-06-23,true,257147805.398,20:57:56.10 +648446334,70214252783782024,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.854,1988-06-12,true,1028748777.75,20:2:18.47 +342288522,15382058403252768,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.854,1995-01-17,true,1345753431.13,9:18:49.31 +1328264196,80810838011732144,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.855,1966-11-24,false,1140719133.83,11:8:47.35 +-246348251,33193876614181776,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.855,1972-09-26,false,148885374.647,15:54:23.23 +1215473270,53349974494043304,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.855,1978-01-05,false,53430489.524,7:10:5.34 +-1271924350,10168972578473810,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.855,1962-08-22,false,696257015.252,20:31:45.35 +-818260675,78491595056511728,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.855,1977-10-19,true,719648744.493,20:13:24.12 +-1498162308,13688418386693898,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.855,1980-05-25,true,395441291.601,4:15:42.45 +-1917425341,43691891485022344,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.855,1982-11-13,false,1111331688.9,1:53:2.31 +432776936,37484667160170264,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.855,1974-04-10,false,440719687.913,17:53:52.27 +-1231681274,86831809537418704,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.856,1968-04-01,false,714842105.989,5:11:21.48 +-1424718556,82751347784246880,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.856,1961-10-19,false,869011922.71,14:22:16.46 +245193671,4548120647081994,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.856,1978-01-25,true,718717730.887,7:42:8.4 +254812462,25673096828850156,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.856,1988-07-16,false,1226853323.06,18:56:30.21 +643626978,39621807836477000,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.856,1957-10-08,false,1201721839.25,8:54:58.11 +334378624,35111094027291640,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.856,1967-07-17,true,1126651345.68,14:17:24.52 +-1088153196,58729616618891872,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.856,1991-01-11,false,257679487.65,8:29:33.24 +600805523,67781106743713792,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.857,1976-07-02,false,1022922404.26,13:25:54.48 +-555971154,16624104969778524,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.857,1954-04-14,false,237144448.667,3:19:19.47 +1413575702,9156688659440098,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.857,1955-10-19,true,154656883.869,22:41:3.54 +-1900645143,84037117553631344,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.857,2013-04-22,false,625433202.287,18:23:53.51 +-236124780,19586193169799128,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.857,1998-05-16,true,991813825.281,9:36:42.12 +-1809441759,37139594059311408,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.857,1975-02-20,true,148817037.143,19:9:54.28 +1191027582,17444329455090954,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.857,1997-09-19,true,1003912143.81,9:5:29.42 +1516934491,9161824255115858,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.858,2007-01-01,true,769099722.755,3:5:34.31 +979431660,70346917625809328,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.858,1974-11-01,false,798513484.847,9:50:24.8 +-948460031,53788607651524360,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.858,2000-05-21,false,451116866.166,20:35:6.10 +486669583,46287366864528536,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.858,1968-10-15,true,388528041.178,14:51:44.35 +537017310,45494810630658472,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.858,1992-09-20,true,1366811062.19,18:58:17.3 +679225532,50280289901998960,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.858,2011-11-23,false,519720272.766,7:57:31.24 +1856566350,19223070397694852,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.858,1952-11-04,true,475266716.254,17:22:15.34 +2145493300,7527981402982625,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.859,1978-11-26,false,95850841.6939,18:27:19.33 +-624521155,76700172610007232,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.859,1992-02-10,false,1123168716.74,21:49:38.25 +1928381682,44577452596493808,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.859,1973-04-06,false,1271659606.43,11:54:37.26 +1432140854,28859335898178468,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.859,1955-10-13,true,830340754.773,16:23:40.41 +1264948023,55341476756502072,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.859,1987-09-15,false,393971181.632,6:19:16.45 +-731640450,82841213587151488,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.859,2012-04-04,true,721036829.985,9:43:36.28 +-856185957,19195496703855392,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.859,2011-01-24,false,80773287.2112,8:15:55.11 +2034452776,87174612522594416,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.860,1993-11-13,true,690138399.336,11:9:9.53 +-1286320676,50226405862394112,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.860,1998-07-20,true,1364053319.32,15:14:49.53 +897855570,71082227927421144,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.860,1986-02-22,false,76736933.9693,4:13:41.20 +1741824388,82291383689618496,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.860,2002-04-27,false,140839908.548,9:20:3.37 +184457966,42907675237538336,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.860,1988-01-05,true,864817940.379,10:34:45.28 +1122152894,37894376079466824,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.860,1952-02-03,true,716716966.38,3:39:14.28 +-1202469866,78537381103084624,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.860,1980-08-09,false,1073686734.83,13:48:25.23 +1839020317,91246200136973520,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.860,2012-07-02,true,982516833.837,21:44:1.8 +862055351,42049779753114040,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.861,1960-06-25,false,1412727341.15,4:5:44.24 +-1752787162,62366461365913128,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.861,1950-09-03,false,321392467.219,5:2:35.51 +-499436159,87087018510004992,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.861,1956-03-01,true,994095695.421,1:12:4.27 +-747452282,67126928421219264,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.861,1952-05-15,true,346720145.467,1:9:28.16 +1938669768,69287491658720352,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.861,1999-03-20,false,830342409.542,6:26:44.25 +1129220125,14801945032961772,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.861,1957-07-12,false,257927462.604,13:1:34.8 +514565070,29308876645561784,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.861,1991-02-05,true,855951980.668,1:56:33.10 +-458239461,43542982761056056,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.862,2005-06-18,true,102154489.464,8:7:30.14 +-1847305744,12725122939173662,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.862,2006-10-09,false,37604678.2122,21:21:35.6 +-389925345,83855117096137456,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.862,1952-02-18,true,149648408.439,19:22:51.25 +-1200565071,63133947822631536,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.862,2010-01-03,true,147530538.358,22:3:16.36 +-1068416308,85769874361806288,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.862,1960-01-23,true,1006332041.97,10:27:12.37 +1194211492,21410353949168864,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.862,1997-04-14,true,988539299.686,21:32:57.1 +1769324038,30533969377926740,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.862,1954-11-04,true,1311251934.08,5:22:30.57 +1967594206,81994949937727696,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.862,1999-07-20,true,1038437965.51,17:51:17.41 +368466701,58435541759038200,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.863,1984-09-09,false,294506919.013,6:46:14.31 +1006765026,59753745487558832,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.863,1960-02-14,true,161823569.357,16:22:51.58 +1969233043,48406983570410280,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.863,1958-09-03,true,719912027.199,4:12:15.50 +-28596136,47446860207229656,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.863,2005-09-02,false,415520867.351,20:46:16.45 +531059990,14964990683419842,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.863,2008-10-03,false,1190165049.3,8:27:50.45 +1859733542,33971639028177728,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.863,1966-07-12,false,429399277.703,14:27:10.53 +-1119098844,21363752578232496,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.863,1985-04-13,true,79722980.3241,19:34:26.23 +1192792859,28084566326566104,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.864,1964-06-10,false,1108782815.06,7:22:13.6 +1092946065,29578279315344824,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.864,1981-06-13,true,337032474.723,6:47:50.51 +960819899,90456552127255248,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.864,1992-09-15,true,896817440.828,3:14:53.23 +-359506495,41066815454098200,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.864,2007-10-15,true,1144730135.33,5:43:35.51 +1117991891,59033490015668432,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.865,1999-09-19,false,89235941.3445,8:20:7.25 +-1862683620,91799472501313680,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.865,1998-10-04,true,1158388789.9,4:49:9.41 +784514369,54076927501002968,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.865,1950-02-01,false,472192754.165,11:45:10.20 +-1817612271,76042970689143712,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.865,1969-04-19,false,798044169.35,12:5:3.46 +1305959897,16206989059327314,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.865,1988-06-22,true,524013377.13,10:26:56.4 +-104449866,42588688109813352,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.865,2001-08-01,false,1270323439.02,2:27:17.21 +2094868883,62813231287018712,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.865,2000-01-19,true,1205043720.81,11:51:43.41 +206899289,81648203684420384,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.866,1988-03-14,false,343364222.185,3:58:57.26 +1089605821,9415274028271206,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.866,2010-05-03,true,553542058.738,1:46:40.25 +-1549539453,74615748548053312,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.866,1958-06-15,false,825164429.33,6:46:56.11 +757751461,77710401653007504,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.866,1977-06-26,false,380581903.025,21:53:54.5 +238800227,7389075428577260,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.866,1991-11-09,false,1412884890.41,11:3:40.12 +1649159484,59550265620276360,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.866,1956-07-09,false,58134795.3908,5:55:28.43 +305645571,62833070389102736,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.866,1970-11-15,true,507520299.054,4:22:13.1 +-394382726,5350741131657063,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.866,1959-07-07,false,1056562069.53,22:29:44.9 +1819585638,15022349944781824,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.867,1960-04-27,true,1327863357.46,10:28:29.10 +1253298440,71410877934659760,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.867,1958-08-27,true,1057684181.74,15:46:28.47 +-1618646599,13446559946661838,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.867,1987-03-07,false,554964554.439,4:54:19.15 +-1622419979,17877045660637390,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.867,1978-06-06,true,5533621.44212,9:31:10.17 +-91358748,3247850347001487,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.867,1995-02-05,false,695550397.098,21:6:39.13 +-1148304438,15046744451830712,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.867,1977-07-01,false,1428711761.99,7:43:19.23 +-1006292692,60587883195254152,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.867,2012-09-21,true,1422184809.91,12:55:37.20 +1673051114,60317198496445080,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.867,1951-06-27,false,670881808.362,9:25:45.54 +-2035937632,62607465659870248,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.868,1978-01-05,false,353491280.797,13:13:5.36 +272953496,40773571248600176,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.868,1970-02-02,true,975294711.964,8:47:53.56 +54938602,91319738026628480,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.868,1961-03-17,false,612849172.361,9:8:37.32 +-2101966270,61101150684265616,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.868,1969-05-20,true,922361828.697,7:56:26.10 +-1210996883,74121820509275952,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.868,1996-04-25,true,1041769610.54,4:18:56.57 +-799775416,26438246776354244,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.868,1963-04-14,true,866119260.274,20:55:4.45 +65976640,26404191670511288,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.868,2008-10-13,true,76694508.27,3:28:23.18 +1071643549,78803200336139072,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.868,2009-05-18,true,981352608.027,17:52:55.5 +-1943121171,69006428771399200,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.869,1977-01-12,true,738851764.086,18:25:33.23 +-965638654,54439898399040440,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.869,1976-10-09,true,54802645.2277,18:15:43.48 +-1301367848,25128191756043788,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.869,2005-06-04,false,1104630402.62,11:3:3.45 +560683305,76059225819702064,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.869,1986-08-12,true,1226480319.28,3:47:12.36 +-1146515796,33794989184533288,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.869,1978-08-01,true,541600143.174,21:47:48.38 +575409845,60327600315251136,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.869,1994-04-21,false,592635224.372,13:39:5.34 +1861407794,33580741889922016,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.870,1999-01-07,false,1263368383.13,2:1:30.8 +1318693613,21797656548337152,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.870,1963-07-27,true,343059175.813,14:13:15.41 +-1679080607,28878363145927128,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.870,1969-07-15,false,863556220.172,18:28:11.8 +1954598449,31632722006313400,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.870,1988-05-01,true,39219108.3782,8:14:28.25 +394703346,41563867680913704,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.870,1994-05-02,false,928522662.134,2:44:18.35 +42547156,42374208216841936,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.870,1988-02-06,true,1250630148.14,21:8:11.38 +1477000769,84284752144186096,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.870,1996-07-08,true,1021716790.38,5:37:13.31 +-1654998213,87701557149329632,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.870,1975-04-10,true,1194243332.66,9:5:20.18 +13056009,83908979794969312,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.871,1953-02-18,false,1170586271.71,2:16:3.9 +-1983456818,25481321064934452,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.871,1987-01-17,true,187863429.21,22:44:4.23 +199784546,75542584452769520,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.871,1996-06-08,true,493827174.938,14:1:46.39 +1615105472,38904463094031368,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.871,1975-11-07,true,1202694119.41,22:48:40.43 +-801944419,22883742177668628,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.871,1978-09-11,false,1173818860.37,8:35:15.46 +-1590822081,54971042430012368,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.871,1993-10-01,true,1172712808.42,12:32:31.1 +1628426343,16709921747723592,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.871,1974-10-08,true,183209799.413,1:30:27.32 +563945500,71442119160793672,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.872,1974-02-01,true,685064578.699,21:55:36.33 +1321898481,81754985865366752,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.872,1997-01-20,true,653194841.375,16:19:38.48 +1180384499,39674663119547256,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.872,2006-11-18,true,1380473180.55,19:15:30.50 +-1065396257,29867502348218472,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.872,2010-09-21,false,1334452214.13,7:30:55.23 +-237197533,2214523451061452,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.872,1988-03-07,true,416514630.91,20:38:46.10 +-693542609,73600264764684544,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.872,1999-08-13,true,567372966.452,17:6:6.32 +-434497126,18031544293469296,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.872,1995-06-13,false,211663884.415,22:54:24.57 +-198922648,16182959815614474,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.873,1986-08-14,false,470575058.097,9:16:35.53 +-1729610826,27729051974870416,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.873,1966-10-20,true,1131252327.15,8:42:3.29 +1336840950,38965183572752928,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.873,1993-11-14,false,1261486266.22,14:58:8.1 +-1786531138,16171404637420994,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.873,1988-11-06,true,211192148.162,10:10:16.26 +980134917,9117949379023124,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.873,1959-04-19,false,1382811823.02,6:53:33.18 +-1511556047,43703725277366056,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.873,1969-06-01,false,1184944011.65,12:21:14.38 +-16443531,73190932921083376,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.873,1968-01-03,false,450505339.622,20:37:40.44 +1927808677,14756302580286300,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.874,1965-01-24,true,354640483.674,21:49:34.32 +1719467573,56184117849416320,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.874,1962-10-08,true,1408398614.96,1:5:56.40 +-465002746,50861137184875488,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.874,1984-02-09,false,282353380.219,15:22:29.53 +-486263299,24195615374736980,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.874,1981-03-09,true,251899455.284,7:1:20.55 +1067175938,60374045470039520,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.874,1989-02-03,false,912849906.309,3:44:4.32 +-2110661446,9282189969553868,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.874,1952-09-03,false,832444525.401,9:55:35.3 +-254298123,61209424137366456,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.874,2011-09-21,true,692971817.396,18:10:53.21 +569475225,37836548185517872,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.874,1993-10-20,false,275519617.675,15:39:46.54 +-225459410,33467554730679400,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.875,1961-07-22,false,184563508.126,11:48:39.50 +-1473113285,34010264840450068,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.875,1986-08-24,true,1315969733.68,6:51:33.57 +-522946493,19412797329292216,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.875,1962-05-22,false,566262563.757,2:12:40.12 +-795951007,41812089843406416,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.875,1992-06-02,true,311484974.911,10:31:58.23 +491479467,66692887674808344,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.875,1985-04-16,false,749057126.421,2:1:14.23 +-274410867,85682683894806976,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.875,1959-07-24,false,731333209.956,3:1:49.2 +883496334,59595940582828312,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.875,1986-10-10,true,1037947107.78,18:48:4.2 +927879140,2289767085787105,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.876,2003-02-25,false,816777428.878,1:44:57.5 +1400691961,79306222817410368,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.876,1994-08-06,false,813813295.734,7:2:49.16 +-116552411,53930773641311360,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.876,2009-04-07,false,1179711696.68,15:44:7.44 +-940597013,67742526684500688,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.876,2007-01-03,true,68542024.9492,3:48:16.40 +-1038282030,35224811981095528,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.876,1974-03-14,true,1376692307.61,1:15:37.24 +1034421505,61085202982373344,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.876,2011-01-23,false,7816253.7067,6:31:29.32 +-1708224375,35616384844796620,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.876,1964-04-08,false,432108476.566,21:48:39.29 +-2123102766,59888132913356216,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.877,2003-03-25,false,1224278215.87,11:39:29.14 +323163123,73460239883553504,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.877,1976-11-19,false,795926490.976,1:37:55.23 +1297543722,55984266354639824,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.877,1966-04-14,true,578361995.649,11:31:39.5 +1516085815,57350694071494584,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.877,1982-05-24,true,1325813689.57,8:23:7.43 +-1679074188,22572473889750428,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.877,2001-08-04,true,316690568.196,1:18:53.52 +-1118191639,28228134204607900,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.878,2008-10-11,false,1182171241.71,18:31:7.42 +-1336593280,83167165106251408,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.878,1989-05-20,false,325207165.02,16:26:25.43 +-2009907721,74611012041750864,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.878,1951-09-20,false,413314696.799,18:21:3.32 +1389810510,55814542012032216,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.878,1983-04-22,true,715075025.006,7:32:29.1 +194590367,55198941587435456,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.878,1966-09-17,true,799034483.977,15:42:28.51 +-1113124521,22224914797244876,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.878,1959-09-08,false,787802328.484,16:49:8.53 +-583105556,48593848573051192,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.878,1987-02-07,false,445981229.85,16:19:2.11 +-1753544613,51647472132549984,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.878,1956-05-18,true,1009032152.26,14:39:33.35 +-2015196443,75267401447045856,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.879,1996-03-11,false,86312417.6804,6:15:11.26 +885087598,55372822300441896,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.879,2001-01-05,false,1194211735.15,8:17:26.49 +-1430560886,15716607601229752,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.879,1995-08-07,false,48858849.3271,9:57:36.32 +-329222393,12937757648775630,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.879,2006-06-01,false,1238629354.31,19:51:26.52 +-1907733630,49060433783366136,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.879,1964-04-18,true,1013361332.69,6:31:21.35 +1980028251,35114558744899328,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.879,1977-04-15,true,1289214799.22,9:10:36.32 +846831212,88694486470252256,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.879,1974-02-21,false,491709893.013,13:6:6.45 +-846905112,28793938147081348,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.880,1987-05-15,true,1307190620.06,7:19:9.36 +-28390919,10512036805922612,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.880,1977-11-09,false,1154079513.75,7:1:40.24 +-71523242,85550496138310304,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.880,1990-09-21,true,861745501.068,3:42:31.36 +-860654606,23976703174203944,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.880,1981-11-14,true,267069132.501,18:16:20.10 +359079996,22640623561669552,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.880,1981-09-03,false,765670222.885,3:57:53.1 +-1025504973,89571237573715344,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.880,1957-10-09,false,964871577.535,9:12:21.5 +-1231425556,34900254493686704,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.880,2002-08-04,true,1318919398.99,10:15:48.43 +419116595,54314097031837536,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.880,1959-07-05,true,462309089.238,17:29:25.36 +463472220,20220848148956848,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.881,2009-06-22,false,669280396.115,12:50:27.12 +-1903632032,33166083204513608,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.881,2011-02-08,true,424930533.929,3:23:10.28 +-1146584696,88711489904733776,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.881,1982-11-05,true,648155155.405,10:35:26.26 +-788470107,56889469600385744,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.881,1983-06-18,false,445813377.745,17:53:43.16 +1094778928,50365619439911920,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.881,2009-02-09,true,407707981.26,21:5:37.35 +1048787709,79638823844666736,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.881,1965-05-11,false,1373455663.01,9:39:48.56 +1970351598,6874691222405724,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.881,1955-02-06,false,137917667.497,2:29:38.38 +843413474,9845586974678620,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.881,1996-03-01,true,446874810.336,2:25:31.41 +1542434711,85855865645347280,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.882,1962-04-14,true,670527297.45,7:1:30.27 +-1536295375,41591324861354688,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.882,1965-10-14,false,1003284770.04,14:13:8.47 +-1665525786,71095490242107320,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.882,1995-03-18,false,377244028.534,22:37:1.45 +233447570,82701782444643568,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.882,1973-07-15,true,638340072.171,21:41:8.26 +1337533192,24439820351991840,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.882,1998-08-02,false,502557940.194,11:21:51.24 +1237308753,39203677216433888,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.882,1952-08-24,true,816229425.635,11:2:13.10 +1615132834,52044874557143000,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.882,1954-05-11,false,1172627095.92,1:2:49.27 +-1528848160,61955602334538128,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.882,1970-04-06,true,194443225.86,2:8:33.26 +-465137686,58088490703849464,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.883,1974-08-09,false,571830108.268,19:48:10.20 +-231208905,39104879930168440,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.883,2005-05-27,true,421808245.588,11:57:23.56 +-392817871,31954069370077288,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.883,1981-08-23,false,1363036244.66,10:58:1.3 +410500345,24693530975684792,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.883,1964-07-03,false,995787991.155,2:43:20.15 +-841878975,3736462891933880,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.883,1984-07-07,true,1194883096.46,7:43:17.33 +-186086764,64401306861269072,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.883,1986-06-21,false,603194552.745,10:16:50.5 +-527397239,26267012310023076,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.883,2006-10-05,true,1137697516.83,4:25:40.43 +773851649,71662413002109336,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.883,2011-06-21,false,1200271022.28,1:35:52.52 +-1026049607,75980923362778928,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.884,2012-04-16,false,1383220675.73,2:41:34.43 +-573212738,76860413636803824,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.884,1975-07-14,false,381839512.71,8:7:42.6 +306547399,86179967597463888,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.884,1994-08-27,true,106557784.176,1:50:35.14 +991296420,49138152675183472,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.884,1994-11-04,true,751632517.739,9:52:34.29 +-1151256631,75474124645358432,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.884,2013-01-11,false,183960248.592,2:55:19.55 +-629849530,37202833454365984,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.884,2013-09-21,true,955706854.342,10:57:1.17 +-2096802504,39360751347727880,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.884,1953-05-23,true,884366979.51,4:46:8.4 +-163216650,40297234037812504,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.884,1989-02-27,false,1269287915.93,4:7:36.34 +483596635,42040003124907288,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.885,1956-09-07,true,641361256.292,14:24:27.39 +-612120653,24148007210010728,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.885,1984-10-27,false,1018085600.83,19:6:2.1 +509137116,24230101890260008,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.885,1984-08-26,false,298710339.454,20:50:40.43 +154175337,49067920923819760,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.885,1980-09-24,true,935607139.934,12:53:35.51 +-440396913,33095734215189076,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.885,1978-01-13,false,1430032513.5,22:13:41.16 +2072479201,39940311095090864,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.885,1967-08-13,false,72546667.7119,12:24:15.49 +487733164,41676172375833184,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.885,1992-11-18,false,534153224.029,21:31:37.22 +-300841517,2587301019614535,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.885,2008-09-17,false,867414165.253,8:2:25.47 +-590046583,70257159781661584,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.885,1998-05-26,true,693364158.144,22:15:38.44 +-132626175,8472641911614526,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.886,1996-01-26,true,935708911.189,17:27:27.30 +-1613406158,31699833760368712,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.886,2010-04-03,false,1390904605.48,1:37:41.39 +-1951920118,90129891593313360,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.886,1954-05-13,true,130996009.434,7:25:51.20 +-245704118,35473433856949904,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.886,1996-06-07,false,241710021.582,14:46:51.13 +-1239370807,89044563028461424,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.886,2007-05-18,true,1218245237.82,10:18:58.56 +211713798,72395112572684928,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.886,1984-08-24,false,1319655713.27,13:14:51.58 +-765688089,57355473304728888,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.886,1979-09-04,true,77344318.8798,15:7:57.24 +-1621852380,41235867804249752,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.886,1997-03-09,true,175688324.483,13:45:58.6 +-251882163,35315782292447100,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.887,1981-03-07,false,1382715148.77,3:5:2.58 +-1432825899,30694570702489988,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.887,1974-03-06,false,159771124.57,8:40:20.25 +1827582447,2529602274376171,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.887,1950-09-18,false,221790898.116,21:47:36.54 +1038537287,53783056221778464,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.887,2013-02-01,true,260671713.121,6:58:3.35 +86354998,12466740705594204,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.933,1970-05-17,false,787317150.679,22:52:11.46 +1526491104,70140406316316984,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.933,1977-07-08,false,994402993.335,3:58:25.12 +104284895,64058473153640472,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.934,1953-09-02,false,1365626085.74,13:39:2.32 +1112985884,43775483766623152,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.934,1962-02-08,true,605604504.948,19:8:12.58 +334199238,21774609674256540,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.934,1953-06-07,false,91454427.6425,12:9:30.34 +1779064218,58456355676230040,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.935,1969-10-14,false,335317783.792,5:20:7.54 +-1541963075,86713651973235680,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.935,1986-06-27,true,114132304.308,2:36:39.57 +-1620088683,34013180673506612,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.935,1978-03-02,false,411323180.013,5:38:22.37 +-185607185,55654760802784976,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.936,1970-04-24,false,386260759.262,19:34:36.27 +592721687,27031878055333928,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.936,1959-02-02,false,1410432347.23,7:7:23.35 +1250435325,64508648157029584,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.936,2012-01-19,true,226308378.311,8:53:17.35 +575177766,32394384812685200,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.936,1983-09-16,false,373717504.687,9:57:32.40 +309528403,37264081387381984,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.937,2004-11-10,false,1248288770.53,6:46:23.42 +407417386,4318741459355812,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.937,1971-11-19,false,1383718475.99,3:51:9.13 +-613009771,57855823702087264,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.937,1963-08-21,false,776179352.224,5:55:26.36 +-1298756502,8072862394483200,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.938,1984-11-04,false,434015865.34,1:26:1.47 +1795467898,12008485703108486,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.938,1966-02-05,true,76675628.607,22:15:39.26 +-431138401,38177971042046584,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.938,2003-01-18,false,581099690.33,16:56:21.53 +-143158482,86490095438316432,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.938,2003-04-20,true,625928473.4,5:33:16.22 +-1022735695,86283826245578448,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.938,2011-09-09,true,848664864.072,4:4:23.34 +1143668483,40322390248449288,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.939,1986-01-14,true,591592788.431,20:6:22.37 +-1544649120,36433913528475032,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.939,1969-01-24,false,20626059.0487,6:27:57.42 +1620231830,11751530099880714,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.939,1987-03-20,true,1278632429.81,20:19:48.54 +-1325136404,58830223019265648,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.939,1977-11-04,false,685011653.482,2:46:52.28 +-899428595,9625491207376568,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.939,1965-10-25,true,336538878.904,22:41:58.35 +556740644,66609037265447560,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.940,1955-03-04,true,1026023892.65,8:41:34.19 +192630080,54759432797230304,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.940,1987-08-21,false,467365550.533,12:38:51.34 +561895698,83132956098453056,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.940,1993-09-07,true,617859942.066,2:16:37.47 +-741528200,69423884978599240,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.940,2006-08-05,false,1341355482.91,9:49:34.2 +1791419758,82343658356626000,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.940,1987-05-20,false,1047400662.42,14:19:27.40 +1583445591,34060982895515968,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.940,1980-07-06,true,1139638029.6,2:43:6.43 +241582113,24328501884996844,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.941,1991-04-01,false,518843349.864,18:15:2.26 +50310782,30294156135553700,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.941,1960-10-18,true,1112317203.49,18:6:37.40 +-1784750249,40476129567397376,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.941,1992-07-17,true,776341527.876,6:28:10.23 +1635400904,77151211462392768,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.941,2007-09-20,false,615870213.17,3:17:17.17 +558969662,20318361211854604,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.941,2009-06-16,true,939704450.988,18:27:6.14 +693576237,30501372898923880,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.941,1958-02-05,false,1351301990.7,14:46:26.19 +-1111150057,79528713964906288,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.941,1976-03-19,false,1088504784.41,20:50:23.52 +-1472724897,4067403771849441,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.942,1969-03-16,true,1337162931.67,16:46:36.36 +-651134966,53657910897101296,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.942,2011-09-27,false,59460484.4558,17:55:20.3 +1798454737,26949967568916512,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.942,1993-08-21,false,1118570942.05,1:21:29.4 +1495503007,5000554266299812,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.942,1989-03-02,false,1399720893.06,21:1:33.6 +1240581226,48968411507026096,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.942,1981-08-24,false,227510783.184,7:26:40.30 +962851474,80270814906048352,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.942,2010-09-07,false,789285004.427,8:27:17.49 +2114161997,75173280317198032,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.942,1952-05-07,false,25396460.5722,3:25:15.45 +-666343124,48579671370413576,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.943,1956-07-10,false,507975606.11,3:28:39.21 +-1633980751,45480958742936328,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.943,1984-11-07,true,904032128.209,11:10:50.19 +181342484,49679798958049200,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.943,2004-06-03,true,1342118531.44,15:33:7.39 +-1623400292,44257661131767656,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.943,1997-10-23,false,766791159.71,11:36:5.44 +-1287248415,64395394000597856,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.943,1955-06-16,false,220448254.506,13:8:6.29 +2057924726,49779828996567976,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.944,1969-06-17,false,1165005104.66,3:48:29.48 +-202418435,21200161059080336,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.944,1970-11-22,false,277695269.24,21:26:24.36 +-58495558,33269909293914644,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.944,1952-08-03,false,813930428.131,21:39:9.53 +-1090930418,14600334000833096,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.944,2009-09-20,true,1115509021.39,11:5:35.39 +-431630870,38518756677205280,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.944,1963-03-15,false,122809016.554,13:31:26.10 +-1597975799,17389804570922016,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.945,1983-07-14,false,748986814.256,6:13:26.34 +-855292739,59999443982107320,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.945,1996-06-21,true,523020035.449,2:52:16.38 +-951139272,55060954493858544,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.945,1958-07-25,false,142929030.958,22:39:17.9 +-1589822727,25372624240088260,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.945,1970-01-22,true,601719028.131,8:36:3.48 +2118745118,64589096609964656,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.945,1990-04-13,true,896073440.834,1:20:31.34 +128502999,14003992145521788,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.945,1982-03-08,true,549515805.109,8:51:56.2 +-136421650,39260423323842512,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.945,1968-05-04,true,913081621.705,19:28:8.35 +-252598298,71040844958847368,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.946,2004-08-10,true,1319684072.67,22:45:42.41 +1787478986,48302100826126392,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.946,1951-01-02,true,824206026.928,14:3:2.14 +-337799405,42009107471576760,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.946,1977-09-24,false,1302388239.61,11:48:50.53 +778114726,89438051695862976,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.946,1960-06-06,true,774475172.017,2:10:12.58 +-462045406,75640781091243712,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.946,1995-02-04,true,744021693.976,6:16:47.30 +-1987271158,32881991753980336,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.946,1970-11-24,false,1292861180.08,14:6:2.7 +419456254,70916725096481536,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.947,1971-04-01,true,540933643.465,3:20:8.8 +1365695084,72203479314836304,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.947,1955-09-23,true,1205364163.97,1:34:11.27 +-908309878,65621803247204688,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.947,1972-07-12,false,968587743.019,2:4:41.40 +-91044282,6322578069348915,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.947,1973-06-14,true,118618093.236,18:8:46.18 +-1509272601,26379145612810784,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.947,1969-06-23,true,690050182.979,12:17:51.55 +1337780027,22506296195030600,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.947,1989-09-08,false,324572564.379,22:28:6.40 +2048432775,86494633345383808,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.948,2006-02-22,false,861275106.712,11:53:33.27 +1179773433,46935055492413072,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.948,2012-02-10,true,717076250.32,13:12:20.53 +1329141268,72445051858844464,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.948,1954-02-06,false,1324174768.16,15:2:2.15 +-391902059,59424532518270520,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.948,1981-02-10,true,1367968533.14,16:31:19.8 +-746962902,8232769578859950,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.948,1972-06-14,true,421432243.855,16:35:15.49 +-559517343,11980596863143004,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.948,1986-08-01,false,1210794051.68,4:49:54.48 +-726410362,57349055654033912,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.949,1989-05-01,false,1272565175.81,6:5:29.27 +884299927,25550537792363336,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.949,1998-01-26,true,872363349.238,14:8:55.42 +867913388,61214332008093720,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.949,1972-03-06,true,109219780.703,8:16:51.14 +-1050522250,53682759366490296,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.949,1998-09-03,false,1138136070.06,11:22:31.50 +464708272,12451851562326600,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.949,1962-11-01,true,795426422.238,19:50:11.24 +254144831,84933702732259568,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.949,1977-06-17,true,656956680.708,5:11:47.34 +59841922,88449522329820384,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.949,1975-07-11,true,563049647.474,16:33:50.45 +-1726024701,50610124888199200,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.950,2000-10-15,true,339618557.387,17:45:43.22 +1189718756,54423920517939536,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.950,1994-08-04,true,938746290.489,3:7:14.6 +-808034030,55686433690107376,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.950,1986-06-12,false,1349630011.01,3:57:6.20 +907342369,16873107949604802,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.950,1960-08-05,true,1273144268.12,17:19:35.1 +1358065981,33925535470136916,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.950,1975-02-10,true,307477054.518,11:34:6.27 +98701006,17045699668528916,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.950,1992-01-21,true,96529446.1671,19:4:58.58 +-1738449573,22328963660549336,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.951,1969-05-23,true,795031402.682,6:43:9.47 +847802199,76747035438940448,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.951,1971-04-08,false,381513229.964,1:11:24.25 +-667893065,77566931214825824,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.951,2004-06-21,true,522571711.248,2:10:48.50 +946582105,22569584928508376,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.951,2012-08-06,false,720594827.95,20:13:19.10 +815051715,50954763977582632,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.951,1978-02-11,false,972072302.318,10:50:38.35 +1964569285,37700643177057800,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.952,1994-05-12,false,1290068510.06,20:37:4.27 +-1547318164,36581612387095616,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.952,1996-05-08,false,505802821.847,13:55:2.55 +1091851547,83691701185484368,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.952,1988-06-15,true,1316328512.34,16:40:29.43 +-339587940,64407960799243112,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.952,2004-03-04,true,1252063641.99,9:20:35.35 +-692575106,40956286196829264,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.952,2004-01-21,false,112796801.232,18:6:49.19 +1393126978,33369462479821916,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.952,1971-10-23,true,438207693.512,8:53:37.18 +-1089888360,50307175218739048,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.953,1958-10-13,false,652142883.058,6:45:38.44 +1118380952,71421846069885464,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.953,1959-04-22,false,90936982.1685,2:31:49.43 +1303573978,90775893731467808,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.953,1995-07-03,true,964857118.523,10:12:22.43 +-1268632078,17853602464887132,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.953,2010-04-26,false,591547758.741,20:17:11.37 +1904401661,23103200792271556,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.954,1953-03-01,true,167461332.444,22:57:17.3 +498801494,1891176095907266,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.954,2010-02-15,false,75243576.3664,12:19:29.54 +-8725218,87963725664198848,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.954,1950-07-21,false,1008230079.6,8:56:25.23 +-847752837,45653172743778264,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.955,1990-08-08,false,111134354.429,11:56:6.3 +-1767165943,18241346757001768,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.955,2001-08-25,true,1102653926.02,3:2:27.5 +1520866932,57628824711740560,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.955,1952-04-08,false,164084727.026,19:40:14.32 +-2011495338,40929878786430384,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.955,1970-01-14,false,805871882.03,21:47:35.6 +1458171238,64397056042857904,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.955,1956-09-19,false,966420004.254,4:35:47.37 +302515427,22171190104359252,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.956,1979-03-09,true,1189644691.01,7:3:58.4 +-1447136608,8440709624638177,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.956,2012-02-27,true,1374511981.3,21:27:5.57 +608594215,40930873574402328,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.956,2010-10-16,false,409496952.768,22:29:55.27 +1640272197,61731625109103968,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.956,2002-04-04,true,234220766.802,12:28:51.55 +-2140854421,58694576613513616,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.956,1998-02-20,true,143769051.854,19:31:29.11 +-446063646,47500359085847864,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.956,2006-10-21,true,125528921.238,9:27:1.40 +1556445673,17970445283142902,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.957,1964-08-14,false,522226693.619,14:45:37.21 +1790443094,7473074314948280,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.957,1954-06-05,false,173896091.708,18:10:27.37 +866365591,28383337420866520,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.957,1957-09-19,true,946610372.866,14:19:7.19 +44219197,71181792791214072,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.957,1979-04-13,false,1340433477.84,7:36:20.32 +-287862492,43090853138835424,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.957,1975-02-15,false,88686382.5824,2:29:48.9 +-68615620,80681999897823648,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.957,1970-03-24,true,662624142.613,18:24:27.51 +-1503737019,69637813709092232,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.958,1985-11-03,false,820943776.2,9:11:9.7 +-1530802005,54325443316992688,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.958,1969-06-23,false,1342246953.83,21:45:42.26 +-665065865,62670642157520088,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.958,1986-08-23,false,1067089495.78,13:55:33.45 +-806911201,74165622564811168,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.958,1964-11-12,false,262135664.533,22:35:49.23 +-1407572553,57648642816613736,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.958,1953-01-10,false,385448305.831,10:30:37.19 +-854959136,1933236039396260,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.958,1992-03-26,false,939220220.498,19:16:49.20 +-866875073,37153155551252688,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.958,1993-04-06,true,898079636.054,15:40:14.55 +1630589580,61647910462727152,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.959,1964-10-15,false,641032494.806,3:39:46.45 +32944217,29597077264528496,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.959,1970-01-06,false,1115639741.64,19:44:58.21 +261910326,74647791693939216,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.959,1961-04-20,true,1361478937.62,16:54:5.14 +-1235068561,36878622701536592,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.959,2005-01-22,true,1013169848.71,17:42:25.51 +1022370286,73248583979243408,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.959,1996-08-02,false,1303320676.19,9:12:9.53 +1775415521,50015415285342008,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.959,1981-03-15,true,362428547.192,11:45:12.17 +395944463,85668915480725872,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.959,1953-08-11,true,245513822.59,4:5:16.4 +-1218451320,42393007112960400,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.960,2002-11-17,false,252396578.538,20:33:29.1 +1124152024,11124035091532842,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.960,1967-11-15,true,1183584340.9,6:21:42.6 +169145322,90066940655451104,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.960,2005-06-03,false,1211847865.09,9:30:19.17 +1794451129,54268128645719184,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.960,1959-06-26,false,561939735.649,13:22:24.46 +356618156,73529165200995264,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.960,1959-06-12,false,762107471.034,5:21:33.28 +747538564,60488981278931456,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.960,1985-11-20,false,1150722849.32,13:27:24.19 +-320772187,11866202090846812,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.960,1992-10-09,true,723231358.57,4:12:28.3 +1275028052,70641053031222640,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.960,2008-06-19,false,25915321.8444,13:41:55.24 +650366616,70172183085761968,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.961,1992-08-20,true,1031254253.72,12:38:56.55 +-154884133,49506542073010648,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.961,1968-01-24,true,1354258810.58,5:39:40.26 +449374462,20339996056010004,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.961,2008-11-15,false,473657285.585,1:4:38.53 +420950795,75727790800222960,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.961,2010-02-11,false,1086725157.96,2:56:52.47 +1346589165,8276846247198741,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.961,1996-01-21,false,1385673898.97,13:33:18.18 +-390275380,7164451763126375,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.961,1955-06-25,true,573959135.743,16:15:26.21 +941574092,42823505770065728,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.961,2011-08-01,true,231788988.184,2:58:32.53 +1429634630,1266026397051771,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.962,1976-06-13,true,1004973459.69,2:47:45.53 +883474479,72187243721296768,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.962,2010-05-18,false,998326825.112,2:18:18.51 +117543521,49392932461742928,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.962,1966-11-15,false,1413819019.85,1:22:49.10 +-211647934,85331893732330464,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.962,1961-03-04,false,958483045.902,10:8:44.25 +-1163201555,86722110274497152,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.962,1966-02-19,false,1427555891.15,2:37:46.40 +-1562457998,14045184513693696,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.962,1994-05-22,false,812716136.478,9:18:25.29 +1545314602,4226185954210406,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.962,1972-01-03,true,935999431.206,1:43:41.33 +-2046138638,84902102113624000,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.962,1996-11-10,false,456252646.196,20:47:19.57 +-435372833,64086931007038048,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.963,1979-09-17,false,348404837.801,5:54:50.48 +107261020,5917297851951821,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.963,1962-03-05,false,1148367370.68,18:35:24.5 +-208428635,89641822173258992,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.963,1954-08-24,true,462666668.811,3:58:11.52 +1285174043,74417586292426240,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.963,1979-01-13,false,58976514.6032,22:35:1.33 +-237810661,41484728114836136,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.963,1991-06-16,true,352119059.933,15:37:7.10 +1460033669,70849115613345080,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.963,1983-11-16,true,830967354.746,21:42:45.34 +1534685738,6225098611346596,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.964,1997-04-19,true,1400863710.11,18:43:33.5 +242116183,10255277782296238,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.964,1982-09-10,true,41482695.9436,10:27:56.33 +2089920560,58250414456997312,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.964,2009-02-26,false,593295476.485,17:1:43.14 +1382489065,49126386728031008,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.964,1972-03-13,false,1317642497.06,20:29:51.33 +492495083,84834801206127232,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.964,1964-05-04,false,485953537.643,16:13:40.52 +-1963608534,33092054369393868,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.964,2011-03-12,false,1001699238.46,14:29:22.3 +-1757409533,76557273170505776,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.964,2012-06-20,false,678604031.56,12:21:33.12 +860295408,58096456387979544,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.965,2011-11-10,false,99905643.088,19:40:56.13 +-2085834640,65496708137280512,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.965,1976-05-10,false,123613137.354,1:45:8.4 +-1746821734,48266804168397008,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.965,1951-01-08,true,1008149881.22,7:19:45.7 +1313206496,1024443347645235,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.965,1994-04-25,false,404938991.01,7:56:33.9 +-1966887880,37152602981245216,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.965,2010-04-11,false,381071218.596,13:30:29.45 +1972016814,2154421737918628,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.965,1999-04-06,false,729777278.383,7:14:2.42 +1386006199,61643310888511952,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.965,1979-02-13,false,768804803.298,13:38:5.37 +-1366381244,1216066964746834,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.965,1957-09-07,false,915431702.132,3:15:57.16 +443596464,5259918180820951,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.966,1981-09-02,false,1416197916.22,12:49:14.32 +-610961787,65056500555185616,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.966,2013-03-23,true,18930511.9546,17:57:17.32 +415049720,74711156051403888,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.966,1969-09-12,false,1060717004.24,10:49:17.27 +-981242473,36374241426376048,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.966,1984-10-16,false,450299322.818,4:33:38.33 +1799124494,27768966491010368,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.966,2006-01-17,true,935366418.139,21:50:21.58 +79733633,65909948451906816,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.966,1977-07-06,true,1355677690.51,2:8:34.16 +-269938071,54827730024169200,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.966,1951-03-22,false,1008058497.0,14:45:44.36 +-1210140034,58634472633022432,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.966,2004-05-16,false,278314069.586,4:6:37.58 +-1156440267,2533562965850173,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.967,1964-03-05,true,504968407.607,21:11:10.52 +-850303062,50597571233897944,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.967,2005-11-15,true,1403496554.19,18:5:5.55 +2049126866,55914998177400720,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.967,1979-07-15,false,668451798.274,22:44:51.43 +1099445786,55488972014467128,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.967,1960-01-16,false,230423669.43,19:39:28.47 +1530663463,83710303374778240,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.967,2006-10-02,true,22259910.27,11:51:44.57 +63406690,11593805333985270,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.967,1998-05-15,false,385590376.371,4:36:7.6 +1884213221,48865147621586272,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.967,2000-01-27,false,1400013570.88,5:29:58.7 +-1749806072,91354472018137776,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.968,1979-11-05,true,468382413.044,7:40:29.17 +264430596,81728219866391584,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.968,1979-02-14,false,475923322.223,2:5:19.11 +-1453378044,59189397474089728,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.968,1993-02-03,false,364673634.124,22:8:13.8 +-1319075216,511211379552901,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.968,1978-08-25,true,952795504.154,20:47:15.40 +834421335,83466858725031472,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.968,2001-07-18,false,1416868922.03,2:41:24.34 +-936176663,54876950830817792,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.968,1980-06-07,false,312209565.038,13:34:29.13 +1521346659,25267594399419720,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.968,1959-04-15,true,208868604.984,20:38:35.32 +1613601050,84451276882084640,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:22.969,2004-07-14,false,97075236.4751,6:49:44.22 +582624046,37815979753542840,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.969,1958-06-02,false,504846466.364,22:5:20.8 +-1911957604,75174019568521504,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.969,1972-05-19,false,2639206.1403,17:49:16.12 +103873218,85553440908421632,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.969,1998-04-26,true,1193431962.11,4:51:30.22 +-600105973,12516970921528064,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.969,1999-03-01,true,639985957.374,6:17:45.37 +1210404486,70887386255350992,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.969,1955-06-02,false,1408277594.59,15:48:39.32 +-410329750,79561219456102448,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.969,1955-10-09,false,679046831.163,6:25:3.28 +-1324623026,16223119897039596,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.969,1954-02-07,false,1311879644.44,8:22:5.40 +693894784,68842225329753184,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.970,2007-04-16,true,186779759.958,10:21:33.48 +-550717537,92135603146670560,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:22.970,2012-09-09,true,153623924.864,4:8:5.42 +-2087158467,80519866395429312,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:22.970,2011-04-03,true,186646875.405,20:26:45.49 +-1495024830,52488785591776840,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:22.971,1956-08-01,false,953270816.574,18:14:20.17 +632796235,24667501719364804,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.013,2004-04-25,true,221468139.649,6:44:53.12 +1152014621,68749142297872112,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.013,1972-08-05,true,1417629563.14,12:37:36.26 +142056117,65174313982553008,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.014,1955-07-07,false,718061998.741,7:50:39.13 +-1416190087,71720497764764256,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.014,1979-04-20,false,603828812.373,15:3:34.29 +835404973,27718225244151204,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.014,1953-06-22,false,13590663.4687,18:17:53.23 +1051495611,49858711322121936,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.014,2000-02-21,true,1090993621.37,19:48:20.36 +1608044238,60734519226027304,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.015,1956-08-11,false,432242392.187,10:40:37.29 +1196462657,46153902398287648,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.015,1987-04-06,true,824552930.607,19:18:31.2 +1077107027,22683510875705080,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.015,1972-01-26,true,1351207778.3,11:15:14.34 +-426588113,75112421300472544,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.016,1986-02-18,true,1029005254.25,13:40:14.23 +916752319,2105053121566884,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.016,1954-07-06,true,659277048.351,16:48:27.12 +-1777015027,39282260750064168,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.016,2010-01-22,true,1098796433.23,1:12:45.17 +-1776276590,75679491079279376,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.016,1978-06-23,true,228623882.729,14:2:7.21 +-496909789,54558053944437648,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.017,2000-09-07,true,881560371.652,11:5:40.46 +-121827485,81170208538795584,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.017,1992-03-13,true,112429270.175,9:12:42.34 +1288733671,31665209557902376,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.017,1998-03-05,false,526677628.572,11:24:9.13 +734443626,24522473340121120,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.017,2001-06-20,true,409558079.107,21:31:32.55 +-52161897,21967465113113948,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.017,1955-02-22,false,1040280616.08,16:35:19.18 +-1528657503,82471821371227104,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.018,1977-02-25,false,1165393326.35,2:53:28.13 +-198952138,60391880143540584,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.018,2008-10-18,false,359941990.284,18:4:45.58 +-1414870329,34566116863841148,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.018,1993-09-05,false,935197516.775,19:17:39.16 +-372648215,39617841597124424,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.018,1976-11-05,false,1212308202.3,9:32:9.31 +1721122068,56509721963391808,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.018,1956-01-04,false,950465147.739,9:10:33.4 +-1166132634,41928039033961336,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.018,1955-10-21,true,1181994380.44,15:21:48.29 +697620831,77299953267147968,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.018,1957-05-27,true,745864298.241,16:47:21.11 +-548048126,3404648408720496,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.019,1972-11-13,true,567221049.561,21:1:19.6 +-42040484,9203779000105698,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.019,1980-01-17,false,547198103.768,4:49:16.58 +-1589192261,38986388824687312,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.019,1994-04-12,true,468851082.839,5:16:15.55 +393116229,89772296449681312,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.019,1962-06-13,false,299617404.509,4:49:42.8 +-2012313840,64309649733213840,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.019,2012-06-03,false,384985316.704,9:37:55.21 +-1651350296,65976201707300224,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.019,1957-10-19,false,1374343225.02,2:6:46.2 +2186777,77774903927399936,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.020,1963-03-07,false,10542357.8495,6:4:8.58 +-1745722576,61714801941177384,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.020,1960-03-09,true,1119499337.57,17:31:48.49 +567806190,71684548288675896,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.020,1978-03-17,false,1254580020.23,22:49:38.55 +446748974,72085633888014704,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.020,1969-04-24,false,164030078.912,19:3:45.53 +-516055996,81988565436444944,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.020,1963-07-13,false,1199891464.76,10:47:15.46 +-1871946794,2208867136766689,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.020,1990-02-16,true,995145069.744,4:16:44.12 +1726086518,56831978761237376,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.020,1960-10-02,false,379317513.602,10:42:2.32 +-1125118122,50480093046631904,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.021,2002-06-03,false,527604813.607,17:11:41.40 +95783497,92147119879845824,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.021,1969-07-12,false,981534347.718,7:14:54.57 +-358643243,64927490375918632,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.021,2004-02-23,true,900546562.974,21:7:2.16 +2013474000,9478622507746508,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.021,1993-05-17,false,277604534.808,13:48:21.40 +-1019117096,35402248087119944,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.021,1965-05-04,true,37818161.829,3:50:54.50 +1033405210,79770086364502816,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.021,2010-06-13,false,898742539.443,8:22:57.57 +-526347910,50855267685699296,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.021,2008-06-17,true,27964805.5652,20:10:54.27 +-1965746608,49801929629917408,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.022,2000-06-21,true,813265846.744,10:5:3.15 +-1076077069,81999640081576528,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.022,1950-07-09,false,957526925.32,7:11:56.46 +-399474914,88461505544504976,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.022,2003-08-15,false,967187738.549,14:35:55.54 +-1506984454,7920262773761475,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.023,1969-02-16,true,703188080.747,16:41:1.22 +301415246,29005526545391688,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.023,1984-09-17,true,363266909.159,19:2:8.22 +1611078634,22691803358144012,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.023,1998-08-13,true,1190310729.86,12:12:49.20 +932077357,25019743274194844,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.023,1990-02-14,true,1118698081.49,12:4:45.51 +2022053911,31338411336653344,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.023,1997-05-13,true,639655972.221,1:25:23.39 +-1812039038,52169674151209240,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.023,2006-05-11,false,689035017.443,18:40:50.44 +697610365,33545198975934608,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.024,1999-07-06,false,1055986349.71,13:35:7.46 +-277169312,45885316796997712,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.024,1960-04-03,true,1218721829.9,1:16:47.18 +-1949614331,30421145099725724,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.024,1972-02-09,false,1349516957.09,22:29:55.41 +-945708326,5879755084477205,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.024,1989-11-20,false,1126425215.43,6:12:17.49 +-1519618317,3871161587072409,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.024,1954-01-12,true,284646740.83,8:43:7.31 +1276597444,71823871375688704,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.024,1982-06-03,false,1354718975.59,16:4:47.21 +1814089132,71315283009052664,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.024,1982-07-15,true,425158173.947,21:49:9.57 +-1711876966,3373700056329175,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.025,1982-02-04,true,218280229.823,8:26:23.29 +1705953482,27793265029259072,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.025,1957-04-19,false,1068202537.16,3:26:29.17 +-992686401,2053308863647078,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.025,1989-02-07,false,504074642.348,1:34:34.2 +-247037208,19437324874982196,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.025,1999-01-14,false,574430603.251,20:30:49.40 +-507593556,78716129688246176,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.025,2001-02-01,false,641913924.934,4:38:33.58 +-643240805,12486062548579492,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.025,1991-02-26,true,1213118662.38,15:26:52.16 +2006529582,54970296651845760,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.025,1958-03-24,false,268091864.058,10:52:11.9 +832072179,79571671117017408,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.026,1954-11-18,false,1014892573.23,10:47:32.43 +-1633857046,43526840059533448,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.026,2005-08-06,false,1293064302.57,4:40:49.31 +1546097475,64811069834464264,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.026,1952-11-18,false,1215744639.73,12:39:15.27 +5441506,30393907779701892,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.026,1985-06-08,false,373826780.263,11:22:33.54 +-1411441897,11288186986132204,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.026,1979-04-09,true,280124994.182,11:9:16.8 +-1200031534,9684663616979160,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.026,1981-05-11,false,218477404.355,13:2:20.4 +-1492673304,3045268770569820,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.026,2003-10-21,false,1095301575.13,3:15:39.37 +-568070091,20543017436497724,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.027,1957-03-09,false,571922265.405,22:44:53.45 +2137012872,2755194722045563,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.027,2009-08-21,false,590945187.118,12:17:26.15 +640984653,77747959287836704,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.027,1977-11-04,true,1312746914.58,4:5:18.34 +-721462305,76509000077195808,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.027,1978-04-19,true,1153899880.26,21:8:22.1 +-262685945,55308038328348272,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.027,1971-07-06,false,978215473.915,8:57:53.51 +-1649467740,59371136816145744,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.027,1998-08-01,false,283901355.919,5:15:49.50 +-497584243,72248369352679712,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.027,1967-07-13,false,1241379619.36,2:48:18.7 +245001172,61761984477008632,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.028,1980-11-16,false,1053699896.04,6:5:45.54 +540653678,36704976436811872,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.028,1976-05-24,true,523312229.133,19:24:15.1 +-1906340999,45455349329497888,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.028,1974-07-07,false,471075579.853,14:4:12.21 +-1128538526,71015760920414280,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.028,1972-09-16,true,1334322495.67,2:12:6.6 +2002884939,69731265022910520,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.028,1990-05-21,true,1105490416.28,17:23:14.1 +-919415237,15166173888901120,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.028,1983-08-01,true,254589514.007,21:14:35.33 +-398673957,88084909948397744,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.028,2013-07-03,false,265444458.837,20:20:15.58 +1542465496,40556752476594936,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.029,2002-10-01,true,206108149.491,17:4:50.17 +2137354893,36672743811972872,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.029,1978-06-18,true,203702065.158,9:41:5.54 +1436377811,67369052640947456,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.029,2000-07-21,false,1164667449.26,20:30:7.41 +1270659675,15397578312908668,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.029,1997-06-19,true,477372156.315,20:33:38.9 +-845514726,44115320222266536,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.029,1982-05-12,true,642663872.709,7:51:54.3 +1262880124,8833646132315341,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.029,1953-09-07,true,652296059.411,15:37:24.19 +596060224,31542643138382276,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.029,1991-06-15,true,1223352827.3,21:29:17.9 +-1653912702,62308196810833232,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.030,2000-11-22,false,826761757.569,16:50:35.53 +1434386604,52475586503739176,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.030,1968-01-20,false,1355672069.73,7:57:54.51 +-989022454,3849745984557250,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.030,2007-08-26,false,364333715.065,7:44:5.23 +526146586,4142635437617060,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.030,1955-04-16,false,1278645761.92,15:7:39.38 +-1326960012,51255647468882064,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.030,2007-08-27,false,447496396.099,6:27:40.16 +44776139,59778921605956744,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.030,1955-03-06,false,1361741634.77,21:47:14.53 +-1449729,44751032742105408,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.030,1970-11-02,false,13608574.853,10:55:41.45 +2012713263,32514391373235264,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.031,1987-10-25,true,981328282.355,9:29:8.38 +1009273316,21572983904140044,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.031,1966-01-02,false,623660852.102,14:30:11.21 +299416309,40085243737115672,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.031,1974-10-08,false,788106795.015,15:34:25.42 +1428321710,10917345317538858,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.031,1999-07-01,true,728994575.786,6:2:4.49 +-791531215,24098675581640920,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.031,2009-01-08,false,48618881.3415,5:41:43.8 +-1427348847,8829995175038997,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.031,1955-09-23,false,72811732.8473,22:19:55.36 +-831521684,78253085334748624,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.031,2009-03-10,true,344650867.016,16:44:17.12 +1373499979,57005316471814352,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.031,1960-09-11,true,514854488.392,8:57:32.32 +-418755607,60850840544899056,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.032,2009-11-03,true,1272290992.78,20:18:49.10 +-1502993313,39899368954307944,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.032,2007-05-06,true,1240032639.06,10:38:52.13 +431996741,24470221308292884,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.032,1985-09-11,false,293764938.955,19:44:47.45 +364279825,84429195850902176,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.032,1991-05-25,true,1336017200.01,8:57:20.12 +-294509294,47828154412060696,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.032,1978-01-03,true,785014744.959,8:58:16.18 +878301080,59536382387744136,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.032,1989-07-26,false,758921891.112,2:19:37.44 +897651312,2740914722449817,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.032,1958-03-07,false,1226118195.1,6:44:1.9 +180324927,43928933702420320,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.033,2006-02-11,false,94761459.9235,18:11:48.23 +-271092401,71443617963354064,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.033,1950-10-13,true,480324967.904,3:39:48.29 +1149575602,46466397335555896,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.033,1968-03-17,true,87862280.0857,5:23:6.33 +-949469941,46871304192166504,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.033,1973-02-20,true,1086092296.86,9:34:10.51 +-254376917,32175278942416344,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.033,1965-01-14,true,1091383674.18,16:42:56.20 +-1659346581,43241622154986304,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.033,1994-09-09,false,1166147515.36,5:53:37.8 +-372067160,7146203016571802,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.033,1969-06-12,false,3488901.55254,18:39:39.22 +-245426213,66719953543373608,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.034,2012-04-08,false,1359592708.22,15:16:51.57 +-1557454889,8690547725578527,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.034,1959-05-21,true,1186361416.85,13:2:28.14 +-953086788,40766184153757664,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.034,1994-04-11,true,797631277.271,16:57:51.55 +-591458152,56340985212032896,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.034,1987-05-23,true,1345254618.79,22:58:41.36 +-1199456989,71011301995002520,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.034,1957-10-20,false,275032634.354,12:49:4.7 +-1159974832,15621481647067648,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.034,1994-10-27,false,578777673.72,1:23:28.14 +-1788068570,87237740763485776,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.034,1985-06-27,true,811681375.92,16:52:50.37 +2026724059,30540829892206888,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.035,1990-07-21,false,973775045.135,1:41:25.33 +941353456,69357655445225728,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.035,1999-08-19,false,179074090.181,1:37:42.2 +1731923757,25634511951423948,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.035,1992-09-09,true,763727448.144,19:12:47.30 +384889161,62899623661023784,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.035,1972-06-21,true,1103683391.98,20:56:28.49 +1375319924,65538476225684744,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.035,1953-08-07,true,878857195.157,9:28:48.48 +-2086802875,26930322677357436,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.035,1983-04-12,true,1370762494.72,3:50:34.7 +-1923256966,51806925775194352,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.035,1964-06-09,false,844815308.556,13:40:8.5 +1958682429,42490954647469896,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.036,2000-05-23,true,857544381.012,17:26:43.45 +2136598757,11577205507782104,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.036,1977-04-05,false,88393354.6414,20:54:2.37 +270100799,1465168802523125,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.036,1951-07-26,true,672764242.246,5:23:57.28 +-464365768,89291965710720656,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.036,1954-01-13,false,1343527087.44,9:10:42.37 +1722024028,30171241109205556,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.036,2006-02-21,false,685802516.729,3:24:17.44 +1465671228,1852089773794252,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.036,2010-02-19,true,214840331.457,20:52:40.23 +2015550967,91394829809329936,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.036,2011-04-01,false,654556265.77,7:18:39.53 +1517415462,40955898420054720,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.036,1984-04-12,true,679498188.855,14:55:51.10 +891926981,61008551743395480,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.037,1957-07-04,true,211384233.042,18:7:12.40 +2141589211,36850783274266320,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.037,1990-11-21,true,1303957688.13,10:14:43.41 +-891113790,28028458883399752,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.037,1964-05-18,true,1153949409.01,3:4:52.45 +645947508,73951462772544848,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.037,1950-01-12,true,980573130.132,6:30:6.40 +929385126,30192295340372420,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.037,1974-02-08,true,415797018.9,16:2:47.18 +883259770,16694516312401224,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.037,1957-08-16,true,531826954.498,3:21:56.22 +1471979949,36520236778267928,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.037,1952-05-16,false,1210879086.78,10:34:39.37 +1191120776,33202643379034092,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.038,1965-01-08,true,1037348382.61,3:16:21.34 +735146687,45761158280571384,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.038,1983-01-01,false,931476504.563,4:46:13.8 +1365684709,66177872211624344,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.038,1978-02-07,false,411125703.748,21:51:15.45 +403304174,2707640058350141,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.038,1994-11-15,true,582336452.299,7:42:55.2 +-1934651441,86717752321820736,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.038,1979-08-13,true,1040275312.22,1:24:36.47 +-1372603131,32837432277327840,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.039,1985-09-22,true,502557118.232,19:24:1.51 +-610230798,85979167709785792,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.039,1957-03-16,true,1121756089.67,11:3:28.56 +-1052401758,67784878358253504,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.039,1962-02-25,false,529257584.955,4:41:49.56 +-1700775422,52648644191502000,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.039,1975-04-22,false,56529144.7545,10:9:12.52 +-1141532982,72411782055551312,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.039,1984-08-22,true,601717070.336,4:48:16.3 +-1219835662,28920354994082224,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.039,1981-01-15,true,1085664832.65,11:31:16.25 +1958601920,74174213336748416,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.040,1973-01-27,true,1301930432.9,19:25:41.32 +503912840,71865093279157760,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.040,1954-10-15,true,1007095578.69,6:39:4.21 +-313988256,20824016789458444,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.040,1950-02-13,true,731151800.753,8:49:37.36 +-1333496908,77587177276384848,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.040,1972-05-24,true,345035580.886,4:42:4.6 +356471682,91124899531838032,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.040,2008-04-20,false,538497430.005,12:50:21.16 +2024292126,65517399266601472,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.040,1976-09-10,false,244200355.925,19:22:7.24 +882466463,68188183421980824,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.040,1996-03-27,true,539065767.988,17:14:34.36 +155142422,83116966368511280,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.040,1976-02-13,true,963573607.121,5:54:21.54 +-850301972,91789364862506512,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.041,1990-02-24,true,988089720.564,8:14:8.14 +-775249620,11409731413405646,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.041,1965-05-19,false,1204767378.56,18:12:25.18 +769328768,55372619057894880,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.041,1986-01-16,true,653193477.319,15:53:37.18 +-976037795,1365299362499573,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.041,2006-03-12,false,245848813.279,9:51:1.3 +1728222430,84209483970795696,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.041,1957-06-24,false,167395766.273,22:32:18.10 +822480830,49599554030942520,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.041,1956-01-27,false,1222384868.92,4:36:35.45 +943603335,40049850105063456,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.041,2005-06-03,false,331643211.843,7:31:24.16 +558611272,85328714969520464,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.041,1960-10-06,true,1276956443.47,16:16:28.18 +-1168145284,4090099772864655,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.042,2011-05-22,true,917816371.698,20:46:18.55 +-2075387621,45519008854617184,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.042,1981-09-04,true,1083771014.45,12:49:25.34 +1235826493,77239338758240560,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.042,1968-07-26,false,23219298.4002,4:1:19.47 +1061010156,18749438457945724,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.042,1997-07-18,true,1037658845.18,5:11:29.50 +1722946540,7437605802346455,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.042,2009-08-04,true,794232113.2,20:35:8.49 +-1378298435,36521716823368192,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.042,1957-10-13,false,1219057055.83,8:29:45.54 +1656556523,77152370791923424,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.042,2001-11-01,false,65577590.4995,22:22:10.41 +169786675,22593128338570384,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.042,1997-01-15,false,430956231.195,5:44:43.48 +124801552,62297737985603104,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.043,1950-10-11,true,604287399.403,18:25:22.31 +-2669406,53849834166552496,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.043,2002-01-20,true,80887667.304,7:22:6.56 +-1759737823,82259731141044896,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.043,1999-09-12,false,164677783.65,3:31:5.34 +-23153106,56253640590517688,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.043,1971-01-23,false,856351197.71,1:44:28.9 +-136118588,86204704237510384,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.043,1988-04-21,false,786001191.432,19:2:21.48 +998965035,23160837089800788,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.043,1956-07-11,true,1133252006.51,14:20:41.42 +-387075847,17029696882941390,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.043,1972-06-05,false,222276556.961,14:1:8.30 +1431528430,17064601293996432,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.044,2007-08-07,true,1202708744.22,13:3:46.14 +-534755102,5843467924723354,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.044,2012-04-16,false,875547532.836,19:21:46.57 +225026250,67269221899129968,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.044,1997-05-16,false,168325706.329,19:43:23.29 +-1245422569,31071794672439544,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.044,1996-02-21,false,171885852.761,8:55:9.28 +274030429,63360105440992776,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.044,1972-04-14,true,906222687.949,22:26:13.21 +1776241551,58588048410894208,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.044,1957-09-18,true,1320448604.13,13:25:20.48 +1822493738,55978988868583448,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.045,1973-11-27,false,92705764.9512,22:17:21.54 +1129574210,67885896991305464,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.045,1976-09-19,false,760031160.368,13:55:21.55 +1343117143,13258058998420080,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.045,1997-11-06,true,1236425326.87,8:35:11.58 +1118988424,10441709193422542,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.045,1984-06-08,true,1194757634.58,6:20:28.40 +-35883623,12001755333686518,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.045,1968-06-26,false,703677745.375,2:39:26.25 +-1634366276,2256271830511196,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.045,1985-02-13,false,1015603021.03,5:44:12.41 +-1537894721,61126980883113360,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.045,1993-06-24,true,859290901.209,5:20:41.22 +1602801005,1574648144770242,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.046,1996-09-05,false,573167217.367,18:35:36.4 +2002351813,77362208789366432,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.046,1974-01-07,false,590708699.246,10:39:11.31 +-1744289198,42294120847568248,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.046,1963-03-23,true,1212486873.0,14:55:2.43 +56278781,69566098197692592,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.046,1967-07-23,false,685306346.527,15:3:17.44 +-786046120,9692114779898644,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.046,1965-09-12,true,845769373.355,22:51:1.36 +-1285553238,14350227120171756,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.046,1959-10-09,true,597103384.812,13:36:32.53 +-574516222,84052449419282768,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.046,1980-02-05,false,539906312.787,4:8:31.27 +195884229,60287274232866960,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.046,2010-08-11,false,828803639.654,3:4:41.4 +-1189057631,9897939914018048,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.047,1963-01-18,false,496659602.357,13:15:58.28 +179127686,12217598055012814,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.047,1958-01-08,true,573490488.168,22:23:47.15 +309507362,52634277803882208,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.047,1986-03-14,false,1287002616.53,15:25:23.20 +-2071832418,28560924458603448,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.047,2011-09-05,true,559113640.594,9:24:39.52 +-380657955,86695536203563120,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.047,2007-07-14,false,1273261952.7,11:29:30.31 +1137544879,74013494747823744,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.047,1959-03-20,true,234204178.73,3:29:10.40 +-2106160358,67346569600469248,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.047,1993-01-20,true,687197005.843,12:33:31.28 +1758940145,89078651154435984,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.048,1969-06-08,true,339937562.735,12:19:49.55 +695355166,69805297159429856,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.048,2004-09-04,true,868239674.502,4:32:53.27 +-1689546007,58229857527769768,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.048,1972-11-04,true,845410132.621,16:46:17.50 +-917048511,51444386487995616,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.048,1963-06-21,true,1240311236.75,9:25:49.8 +-308931529,63248599851539856,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.048,1993-09-11,false,238416486.967,20:44:56.56 +1069811756,18488920232018648,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.048,2013-03-02,false,1048209145.92,5:6:42.31 +-1267792748,50315101966198128,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.048,1970-01-16,false,543808841.069,4:6:36.53 +-364798971,73930478979116464,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.049,1992-02-15,true,287757656.439,5:29:26.1 +330707103,13948992306046556,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.049,1954-08-05,true,777909345.935,4:3:48.8 +686899830,61123152564476328,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.049,1968-01-18,false,1171746625.04,20:25:47.33 +-52001823,49626235052230480,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.049,1967-04-21,false,1250404712.92,16:39:53.1 +856815442,9650915374865480,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.049,1987-10-19,false,1400169997.68,1:48:49.2 +838450805,74119821435037024,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.049,2005-05-11,false,1065225596.87,7:45:45.48 +1502713360,39245615249953056,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.049,2006-09-10,true,285383876.365,2:54:25.37 +1352821551,45660723957234728,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.049,1982-06-02,false,1268520322.82,18:28:1.46 +-927531965,86968073229938416,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.050,1979-01-11,true,1012416642.59,11:17:25.41 +-955518302,50925259666304288,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.050,1975-09-27,true,540133703.582,1:6:58.32 +1555065910,38939544777002592,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.050,1998-09-21,true,1347252992.59,15:46:18.14 +1087935474,35984405643578952,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.050,2011-03-21,false,867683812.32,22:44:37.35 +625592108,90491940890422848,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.050,1999-09-18,false,1429630481.03,14:29:53.20 +-435323096,60900728992867288,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.050,1970-06-06,false,935814483.876,9:45:5.17 +-1617732656,36243179360350528,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.050,1969-05-05,true,29890472.5937,7:3:43.36 +-997202050,9592456053753118,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.051,2003-01-15,true,49678557.919,10:42:18.27 +1150231235,63213180646517096,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.051,1968-07-20,true,926643296.854,3:55:21.19 +876971600,46144132642374872,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.051,2013-08-19,true,354067040.23,20:36:40.25 +-1621638007,12604594014165464,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.051,1979-11-07,true,703934664.875,16:43:20.6 +1151479085,83151519331322960,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.051,2002-03-09,true,926397013.411,1:6:47.12 +661191990,16529446437360896,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.051,2001-02-13,false,714724821.448,19:56:21.15 +-1467068698,3874336542922301,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.051,1980-10-03,true,203245575.537,5:8:22.40 +-892070621,91281783994937728,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.051,1986-09-22,false,850533741.908,19:8:14.10 +-1213830277,44739787557344824,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.052,1978-02-24,false,596115526.233,11:4:49.26 +-883932071,5500183885054894,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.052,1956-11-03,true,1217283492.97,17:3:40.57 +1248102181,6260862055442412,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.052,2002-01-07,true,823512250.909,3:1:2.1 +1602906704,37116075112151280,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.052,1990-10-21,true,93394364.3896,2:55:29.4 +1030603193,46353204380881856,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.052,1970-10-01,false,118757379.801,13:22:13.9 +1944443797,57406735073801240,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.052,1988-03-19,false,407341198.143,3:36:45.38 +-1061590775,41909368303119280,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.052,1961-07-17,false,406681517.58,19:4:39.53 +67319231,29269594775427380,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.053,1981-06-02,true,405901356.048,9:48:15.15 +816046187,19016030665041676,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.053,1971-06-05,true,87633513.391,18:14:9.29 +-918697657,40132111842554328,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.053,1972-03-22,false,942000677.222,6:17:31.15 +283752503,68269094766253280,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.053,2004-02-24,false,702016748.656,11:44:27.54 +745067066,85987730651044560,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.053,1986-04-06,true,591016244.941,8:13:14.39 +-1263713260,50244013150089432,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.053,1981-10-20,true,918115102.346,3:12:49.43 +-1552968580,80308185917406400,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.053,1983-11-25,true,570718933.234,4:23:15.50 +-964781133,23161645810097356,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.053,1992-09-09,true,876049424.967,10:13:11.13 +-1317801012,77573756189087360,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.054,1976-02-13,true,650381643.034,10:28:51.23 +1243667864,86598622450831376,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.054,2002-04-03,true,1196764401.29,1:10:28.49 +-2036532094,11244243549032100,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.054,2013-06-04,false,1138575942.73,15:2:52.22 +-1051242154,26620887639546664,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.054,1968-06-08,false,618378288.636,22:27:2.58 +-1592667423,27588807901665044,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.054,1956-01-21,false,846172532.703,5:9:2.7 +1870927333,60583788851972032,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.054,1992-09-18,false,486929697.061,13:13:16.42 +129977717,4898855588370698,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.054,1952-01-17,false,502243464.946,13:3:42.40 +875442411,91771741050275632,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.055,2002-01-22,false,905134084.522,6:10:37.45 +-1612157475,16516721160964802,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.055,2013-06-27,false,1387194625.48,20:3:28.51 +-2078072816,70653836557251464,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.055,2006-09-20,true,903471061.878,1:32:27.52 +53718474,13794994348954594,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.055,2000-02-07,true,872923429.968,21:30:38.51 +-1704625573,79889778702171872,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.056,2002-01-20,false,780959970.142,18:16:38.1 +-714377189,21159235584430540,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.056,1957-04-02,false,820655641.114,7:30:39.36 +340571226,2126935672472340,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.056,1980-01-25,false,1189265039.22,6:26:18.53 +1142805603,48409295234718064,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.056,1999-09-05,false,1247080913.31,5:54:54.28 +-1220499905,6058691569957775,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.056,1990-04-16,false,1287568068.86,9:25:41.4 +-1842472777,80607404084332288,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.056,1986-07-09,true,1310377232.22,21:57:10.45 +1960172412,26087574656997692,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.056,1969-04-27,false,900257878.933,16:27:34.53 +-678017941,33414932748658280,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.057,1980-09-08,true,210141537.689,21:28:21.36 +552712317,10285809815350036,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.057,2006-10-11,false,708977004.82,8:8:37.25 +-2032479002,90263676831744976,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.057,1961-10-05,false,425822356.994,7:33:12.16 +-1695822935,68353458746363048,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.057,1953-08-02,true,1050738578.24,2:55:23.53 +-1766629203,6454510157505536,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.057,1997-05-17,false,351617024.633,22:53:25.9 +-2087232727,69008092072363648,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.057,1960-08-25,false,770627302.453,15:1:57.18 +1630947705,34971024281317704,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.057,1982-04-23,false,1294531946.6,20:9:4.52 +1083950864,35205367023254284,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.057,1994-02-10,false,1335616442.46,13:14:45.37 +-1469376911,60631723377690656,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.058,1955-05-16,true,359164855.778,6:48:1.4 +-279615655,48277221529602104,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.058,1993-02-14,false,254744182.399,15:1:52.27 +-1189254895,75616584445273424,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.058,1987-10-03,true,188162702.867,5:29:4.8 +179833271,28302670383465116,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.058,1975-01-15,true,109315020.296,22:32:14.33 +-1552596059,20016462877719368,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.058,1975-03-05,true,259359304.086,13:38:46.11 +18701449,26564764865040476,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.058,1979-07-18,false,563973142.212,12:53:55.28 +-816822004,56986934596477720,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.058,1952-08-23,true,771768070.463,19:37:26.29 +559924119,64500865675757096,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.059,2002-05-10,true,779097257.993,15:44:15.29 +-2070073886,28918652969786808,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.059,1984-06-24,false,1406090280.84,3:51:26.26 +-1983383107,576588037531617,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.059,1973-09-12,true,207066163.846,7:5:31.3 +-1630805792,70559875936486896,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.059,2003-08-05,true,23715605.1676,4:10:8.28 +-246277117,61364652817960376,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.059,1992-11-04,false,433485323.981,19:56:46.39 +-71903882,70880040429867256,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.059,1995-04-05,false,1094249344.19,2:27:18.23 +578483910,90601415275347600,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.059,1984-01-22,true,505869176.998,4:44:47.49 +-877503022,66695653066427576,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.060,1990-11-26,true,367675565.469,13:34:23.42 +-342319170,89635546353008112,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.060,1996-09-07,false,15423229.559,6:17:25.9 +1738124201,51176208721753352,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.060,1989-08-04,true,459403613.045,12:48:48.18 +1121043596,34617341572773704,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.060,1995-08-05,true,519258545.327,14:30:26.27 +-1432147537,47435302907463,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.060,1986-06-16,true,1291620407.62,2:21:38.4 +1050746940,73747891421919584,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.060,1977-08-16,false,1198689869.36,12:11:20.56 +795163421,32529161470264148,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.060,1989-01-22,false,37184718.047,14:16:13.35 +1475621129,38507989423910616,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.060,1978-10-25,false,583136793.856,22:58:18.2 +-1873573490,10393957831370108,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.061,1990-09-19,true,586608087.865,12:57:49.18 +-1612688739,58555911077449976,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.061,1982-04-09,true,93641201.6477,10:22:27.57 +-131968778,54615519591379488,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.061,1960-05-10,true,422676716.104,17:5:38.46 +-1529861782,23796219295699660,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.061,1999-07-02,false,697830049.027,12:9:11.5 +2110135150,78183961336190304,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.061,2000-06-06,true,380175966.104,5:54:47.37 +440755253,72876916628269952,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.061,1979-05-17,false,245916763.264,17:57:12.11 +-1119643867,9930962511922382,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.061,1969-10-14,true,1427648973.68,13:9:37.3 +1551368775,88486339047633360,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.062,1988-11-11,false,419014457.625,18:8:55.47 +-1364801892,63108181094174264,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.062,1981-09-17,false,352657768.232,21:9:10.12 +26195607,53195735812083448,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.062,1998-08-18,true,517779992.006,18:35:51.20 +-775319226,61925546355519240,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.062,1968-08-27,true,630144181.018,18:50:44.22 +383095066,73646713449705104,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.062,1954-08-11,false,1041662924.66,10:48:7.55 +1856884308,38393207129742688,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.062,1990-09-01,true,313153294.83,12:6:48.14 +-1845242637,56720195502528400,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.062,2012-07-06,true,917569842.554,19:8:24.30 +642179691,69234424878030800,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.062,1995-05-11,false,477723236.548,13:17:14.22 +1142522786,90903277524629664,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.063,1956-06-10,true,1159140589.08,12:44:17.12 +-564308418,58286012327390536,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.063,1974-11-13,false,687265014.159,7:40:10.52 +-1547122947,76322310103603376,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.063,2012-08-09,false,531476518.334,15:34:54.42 +373065885,84882004298504480,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.063,2002-05-12,true,486754109.82,13:31:15.17 +-705320728,48895802283845256,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.063,1965-08-04,true,1325258084.82,1:41:32.32 +-313876586,42682570925859800,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.063,1969-10-23,true,414144512.672,1:12:39.18 +541358728,24953337067931004,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.063,2010-11-15,false,687703848.807,13:7:6.5 +-318586479,59213407799741656,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.064,1989-09-13,true,35867538.0893,19:12:12.34 +1817819278,22721272134460476,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.064,1960-02-22,true,103063776.864,8:21:7.41 +1362866569,53786534840453856,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.064,1960-10-05,true,763957101.958,9:48:48.45 +-53593078,77601705764024304,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.064,1976-01-13,false,1399223653.07,10:28:58.8 +-796048137,1580095341195069,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.064,1971-06-16,true,303403164.176,8:16:1.1 +-393444305,50550782908166552,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.064,1965-09-11,true,320149515.978,14:27:27.36 +-734269708,9744222666236682,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.064,1968-03-05,false,795871499.921,10:55:1.29 +2047443874,19372089055870780,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.065,1968-10-15,true,668970947.504,13:53:39.49 +-519756368,50316761453115904,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.065,1955-04-27,false,1182373104.27,6:33:23.37 +-1187437462,12686918962513726,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.065,1976-03-17,true,772066308.651,9:16:6.39 +931708116,14398906844817020,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.065,1952-10-20,false,174598309.816,6:37:54.12 +28628460,40796733497536320,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.065,1985-04-18,true,1230519479.39,14:22:35.56 +-1118193979,40532984150685152,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.065,2012-05-23,false,311467409.776,20:33:49.23 +-68648123,45163488496846528,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.066,2008-02-24,true,638796816.013,22:25:52.48 +684687667,13308069427617444,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.066,1971-08-09,false,273346991.238,7:39:18.18 +-677427739,79577156869107168,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.066,1992-07-11,false,614258930.964,15:20:6.4 +-1244835241,16722510244815534,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.066,1989-05-12,false,162341134.259,19:7:51.41 +1241785492,52629596518439448,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.066,1971-11-26,false,712636985.32,7:3:50.23 +-219000523,75704445574458720,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.066,2005-05-26,true,867533194.02,10:18:28.50 +-1163857338,54063090160674704,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.066,1990-05-16,true,55574106.2198,21:53:11.5 +-209706701,55270980488076864,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.067,1996-04-10,true,21477183.7964,17:49:5.6 +463857492,88029591453161344,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.067,1980-02-21,false,687265405.236,14:37:26.44 +921528978,75756817175720832,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.067,1970-10-24,false,156254965.478,7:29:23.22 +-765154914,34625341931655844,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.067,2013-08-26,true,923409279.52,22:46:21.38 +2096554913,29594757314054880,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.067,1986-09-26,true,1115509958.13,12:32:8.8 +-1007250376,32661444050629652,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.067,1965-06-16,false,1023117318.9,3:57:22.38 +-331507353,67128601332805984,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.067,1962-03-08,true,524005786.202,12:58:8.41 +1397472521,75039952962634656,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.068,1961-08-19,true,693261183.421,2:19:1.12 +1227809083,68360141907246840,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.068,1974-01-25,true,322680706.938,22:42:18.40 +513513461,42394866817205976,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.068,1956-06-16,true,1005856135.95,1:30:21.18 +1532960524,19336829035640012,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.068,1989-10-19,true,775435573.652,9:29:2.14 +1400751281,11371709298567854,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.068,1981-04-20,false,1228060954.92,2:55:32.24 +-2101745281,56813737719026832,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.068,1961-09-09,true,1193752381.59,7:49:43.13 +2117227919,41386415843366720,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.068,1956-09-23,true,855631390.691,11:29:11.48 +-565657279,37645117456048528,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.068,2013-07-15,true,1136677922.24,16:42:53.28 +54420256,11353954814554224,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.069,1981-07-26,true,86807102.0988,1:42:37.36 +374741739,8849964674983045,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.069,1973-11-26,true,268948707.908,5:12:5.27 +-2102588984,17552622296414690,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.069,1964-06-04,false,128954981.641,9:50:21.51 +-1567443684,59094632379226256,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.069,1968-05-23,true,611618149.324,8:47:31.40 +-173676126,30200912206302044,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.069,1986-05-10,true,1339076365.76,7:10:55.5 +-789793066,65150481876275168,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.069,1998-11-11,false,295957337.558,5:25:44.43 +-1355249880,83849363133097568,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.069,1985-05-11,false,1417831260.35,6:2:11.41 +-1972774257,31441638916080424,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.070,1987-11-11,false,1285658846.21,6:34:6.4 +1682095093,14172877558866176,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.070,1969-02-22,true,330502431.317,19:37:58.7 +1981384723,42533365904439944,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.070,1981-01-17,true,507472607.443,4:32:8.26 +-1119483642,48525213642475592,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.070,1992-08-09,false,1190506776.97,2:57:1.18 +1588945620,82247245146053856,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.070,1999-09-09,false,529380862.301,4:3:26.24 +-884079997,36535475953236208,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.070,1958-03-24,false,49035968.7656,14:29:26.54 +1542212035,19003116460491184,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.070,1992-01-07,false,1241695673.3,14:58:35.14 +-829083138,77389600824339360,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.071,2007-03-05,true,1371108944.03,19:38:1.3 +-208593799,39325632221966136,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.071,1988-06-14,true,838754261.991,8:36:51.16 +-1654186674,35729135550664624,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.071,1985-06-25,true,662113827.255,11:8:8.53 +1609700217,65371600712203488,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.071,1990-03-14,true,731985377.201,12:21:5.12 +1702684509,42895904857738872,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.071,2001-09-14,false,656931091.852,21:43:57.55 +-151141767,24387254277036032,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.071,1976-04-08,true,144473876.866,20:32:15.16 +710980119,91921578966897152,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.071,1991-04-03,true,483855164.47,5:46:42.36 +1141159149,77117327633074256,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.072,1961-03-19,true,1416395217.35,2:31:25.13 +-465153667,80628130365284912,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.072,1962-03-10,false,130696567.967,12:20:31.15 +-1386858638,33658617627046544,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.072,1964-03-20,false,854451777.511,8:7:9.23 +1782627471,36349934405506488,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.072,1968-02-11,true,981171538.296,10:56:58.53 +-535605174,39432251085623336,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.072,2001-11-10,false,318192430.27,6:23:19.9 +1470054071,13036932737254738,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.072,1956-09-05,true,872597108.237,14:48:44.55 +503826673,27125378277361100,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.072,1994-02-23,true,838556141.205,15:48:13.3 +-1002784945,42823469897488136,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.073,2001-04-06,true,169609389.326,9:49:24.33 +1451117440,2096379844597412,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.073,1990-08-01,false,373758202.297,11:8:46.41 +-347391107,15508426460858748,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.073,1992-06-02,true,490866002.257,20:11:44.14 +-1799253258,54897611828113464,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.073,1978-08-23,true,480429581.339,9:42:23.56 +263204735,71445732600089528,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.073,1998-03-11,true,1067436558.95,17:45:5.42 +1423339221,53388423498138256,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.073,1992-01-03,true,1143338819.87,19:10:51.28 +1342600819,29481980843955620,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.073,1976-02-05,true,302693709.135,4:23:27.16 +-1035773037,67107236289794712,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.074,1972-05-01,false,516506743.498,19:31:31.24 +175430343,7041916250176737,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.074,1980-01-07,true,216257048.429,19:32:4.7 +633235443,45418610503325768,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.074,1965-11-04,true,66159784.3613,18:36:38.51 +614134083,44453264833205896,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.074,2003-05-21,false,1236883950.76,3:41:7.9 +-438125826,80854003631845360,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.074,1996-05-15,false,923609884.899,9:14:40.39 +-857584843,43843068399628280,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.074,1981-11-12,false,1093429792.97,4:12:12.47 +-1770226179,85133161776934048,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.074,1955-04-27,false,114544964.05,10:1:14.37 +-1550590941,82034121829766288,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.074,2008-07-20,true,1179579457.28,10:12:53.56 +214563003,78118545622592272,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.075,1965-01-16,false,1079233006.35,5:47:31.15 +-1719137018,60431252195375744,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.075,1997-07-19,false,1333034778.71,7:58:22.49 +-903053971,53834520939600784,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.075,2002-05-06,true,675056422.806,3:33:57.39 +1408075307,80715567715716336,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.075,1951-01-25,true,402955730.274,9:29:57.56 +901359352,68549307353308512,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.075,1974-01-07,false,851283271.693,2:4:14.34 +1322507829,67175849717257600,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.075,1966-01-17,false,696261454.187,14:31:17.56 +96493839,87701997494641728,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.076,2011-05-18,false,466872790.97,14:27:31.37 +-1707152000,86350846116711568,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.076,1990-05-23,false,1027034947.26,5:38:14.47 +1702836064,88015135348755808,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.076,1992-03-02,true,283224842.296,11:37:9.6 +-1134760054,73070663934724576,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.076,1992-05-25,false,46858559.4424,13:47:43.5 +354366755,37794853827278040,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.076,1952-06-23,false,1079879733.74,8:50:23.27 +2064373289,64079046444641528,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.076,1986-07-12,true,224146517.175,7:14:27.16 +-973660903,3624685789322742,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.076,1967-01-18,true,522585511.559,2:37:33.41 +70016680,4368489733338429,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.077,1965-07-18,false,25250978.1284,18:56:19.3 +-580930064,88296754579047280,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.077,2006-02-19,false,379584359.571,16:28:9.48 +930014315,40800015624143200,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.077,2010-04-03,false,213973757.164,2:10:46.46 +-1745951113,18788820617662996,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.077,2012-03-03,false,1206277613.8,7:2:53.12 +-1938148418,46783398981183184,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.077,1996-03-12,true,1000806119.2,12:41:25.4 +-615570681,14574086018803886,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.077,1997-03-21,false,243510234.109,16:8:56.18 +-1487201492,82891281997987296,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.077,1985-10-09,true,90379053.2173,14:29:54.47 +1600111544,30194252508747512,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.077,1984-03-05,false,990116067.286,13:29:21.33 +-1540569580,59691366844838840,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.078,1971-05-02,true,516120661.351,11:23:27.8 +-676634672,79210736430579536,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.078,1951-08-19,false,231468281.763,5:37:1.30 +960613766,22979323181954692,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.078,1950-06-02,true,648885267.473,18:3:20.19 +1458727005,33983641657633208,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.078,2001-02-26,true,724651500.141,12:45:24.49 +-1465117340,13479681104465808,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.078,1987-11-18,false,662940000.702,4:13:56.42 +1109755769,53604556234697432,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.078,2011-06-11,false,448041345.057,1:44:46.45 +1992737133,31071751511263108,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.078,1972-07-08,true,435353432.476,19:28:15.33 +-1712639767,33278683300436184,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.079,1982-09-01,true,1035821410.91,22:58:13.32 +-1471613472,10410064625942046,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.079,1969-05-04,true,122995378.218,13:36:27.46 +-94307412,68143628727174384,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.079,1973-01-08,false,360498888.165,13:26:18.10 +-843138981,3940659816243149,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.079,1971-07-18,true,210583404.743,11:34:22.48 +14812472,40817902367662392,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.079,2013-07-18,true,1398239353.93,15:16:15.13 +-2121355006,81844374451267712,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.079,2004-03-18,false,997088068.918,17:55:6.13 +1437842764,21706931086457916,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.079,1976-03-20,false,361014487.265,5:33:20.52 +-1825478966,68155364650773352,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.080,1967-09-05,false,150324458.331,13:33:44.32 +1675564841,45662260838530960,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.080,1985-02-15,false,49774771.932,11:3:53.28 +-1210307608,65071677894190264,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.080,2007-07-13,true,379426975.603,17:37:7.28 +943180086,24223864962729116,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.080,1957-02-23,false,1219767017.56,17:48:48.22 +-1545319722,22500693490391224,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.080,1952-04-11,true,974320263.402,5:27:1.46 +-508471298,20683839767093608,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.080,1951-01-25,false,121514747.836,18:50:49.49 +-1319955223,51708046944873664,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.080,1986-09-03,false,449383139.196,15:21:6.53 +1044187041,49195861633808016,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.081,1963-06-21,false,1106163507.64,6:55:7.28 +-395737127,34677143934657528,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.081,1974-07-04,false,103506640.441,12:40:49.42 +-1951826059,18876979471326464,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.081,1992-11-24,false,1139729858.04,11:11:3.51 +1908551627,4857594171274035,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.081,1991-04-13,true,260206580.544,6:20:33.20 +-583818866,14144650239891220,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.081,2001-07-18,false,395079234.82,19:17:52.35 +-251756657,6342465129317038,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.081,1993-04-12,true,292818683.378,5:32:44.50 +1478728724,15463352622295798,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.081,1980-09-14,true,839434148.537,10:55:43.19 +-1857954631,7558473183900795,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.082,1961-09-10,false,405439859.104,20:43:18.43 +795742282,54211513204171672,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.082,1991-01-14,false,1170485437.33,17:12:33.24 +730163462,70316159846886208,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.082,1985-05-27,false,1308809067.37,18:58:56.13 +565177792,33985377929842372,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.082,1975-07-25,true,879058902.774,6:3:31.52 +1479444775,90973203027588592,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.082,1979-06-21,true,879734582.868,6:15:22.49 +4794009,58387106871946008,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.082,2006-06-02,false,918882392.854,13:8:54.1 +-1327277397,2771168986636503,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.082,1989-06-25,false,610402480.0,17:15:1.24 +-838805135,63120012523170600,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.082,1995-08-23,true,510372948.897,10:34:6.6 +1770762795,18982617776068864,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.083,1952-02-17,true,403101518.369,6:7:20.5 +1265948556,86255297531449232,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.083,1950-09-17,true,868239179.253,22:40:52.8 +-2125522154,56416221549972136,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.083,1998-06-26,true,1291620723.17,16:4:33.13 +-1331658814,85996429985084272,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.083,1964-05-01,true,111125824.154,4:1:50.44 +2015118097,13954829856712244,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.083,1957-06-13,true,9268115.32067,11:47:55.30 +-1340091643,64537806893189688,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.083,2004-10-22,true,689866576.7,20:54:32.52 +-422254260,14999259213670338,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.083,1993-01-24,true,324093624.904,4:43:14.26 +414238597,73958587908692576,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.084,1969-05-09,false,507250995.385,17:15:39.3 +1698242481,70467794116069688,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.084,1968-04-04,true,1145170174.7,3:57:55.58 +140633818,32035550706459588,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.084,2004-09-18,true,664188037.384,16:26:47.49 +706683736,22702177065940164,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.084,2004-04-03,false,1280908881.97,8:55:44.40 +2030959084,65782587499788144,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.084,1956-07-13,false,1055179080.33,3:17:25.2 +726021069,4434730817129851,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.084,1967-05-22,false,241398333.862,11:2:43.38 +-1914310851,74757872550402272,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.084,2013-01-18,false,729916577.268,9:57:54.54 +-423547377,39149282875660784,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.085,1957-10-05,false,1164897595.24,4:43:20.7 +2047751456,15648987574289684,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.085,2005-09-21,false,1240920311.02,4:13:31.58 +-1869385783,86047346474804640,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.085,1972-06-01,true,560927329.631,13:41:38.56 +83016023,46803221499896352,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.085,2000-01-05,true,407922557.662,5:26:43.29 +1835431952,16679263597164462,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.085,1980-09-23,false,1345372548.04,2:52:25.54 +-1156849105,91767729972490336,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.085,1975-05-16,false,1293901395.33,6:6:4.58 +-1697597207,78334415394832016,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.085,1963-07-17,false,1339807786.81,19:33:12.41 +1711329381,3962799364187074,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.086,2011-09-18,true,467735699.417,4:31:6.48 +-285321650,31468255859846604,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.086,1990-02-18,false,1223629924.2,16:57:43.8 +-1268927705,35803874665306952,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.086,1974-09-04,true,1130795330.94,7:30:36.39 +980693752,30304056592145388,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.086,1956-11-01,true,238674239.392,15:3:27.29 +-2038204081,76491393172382224,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.086,1978-07-15,true,931996720.543,18:50:31.13 +-722141430,35906252928877376,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.086,1979-02-24,true,765528959.047,12:49:47.43 +-567387029,48967149354472848,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.086,2009-04-05,false,4546013.25188,7:40:18.39 +650343555,42789644462421440,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.087,2004-05-03,false,533004932.91,19:16:19.13 +-1469831648,44436435110465240,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.087,1951-10-10,true,812245630.325,21:12:8.39 +-1091739751,78075791686283168,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.087,1953-09-27,true,212173727.123,17:37:34.32 +-2090876858,49177050758541792,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.087,1976-01-26,true,354955653.66,6:50:5.52 +-168737773,19476954794723492,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.087,1995-02-18,true,544211258.498,22:28:32.48 +973592457,29021376498633904,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.087,1977-02-09,true,575704713.649,13:38:44.24 +-61559511,35635465278813972,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.087,1984-11-20,true,853881110.039,10:11:28.15 +507631093,10409355032537478,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.088,2011-03-14,false,297223255.439,11:57:15.1 +1304588321,51017137419567032,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.088,2004-06-05,true,84306551.8394,11:1:10.37 +-858099308,4418348458283612,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.088,2000-05-20,true,853635839.308,3:38:36.13 +2103122751,38996004088550648,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.088,1972-07-15,false,1176005403.45,22:57:5.53 +1175282394,8565707583718584,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.088,1988-07-24,true,945279980.133,15:58:10.55 +-118598927,57910862638046960,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.088,2009-05-12,true,732747431.985,3:1:22.6 +-1553909804,58631754276207240,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.088,1992-05-07,false,265026645.363,3:49:7.23 +933356281,89811593700081312,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.089,1998-02-23,true,916840.685186,3:41:37.11 +-1662134760,5364145650916833,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.089,1983-03-26,true,105337213.242,16:55:18.47 +-699691385,42550773313300304,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.089,1971-01-24,true,564493882.005,21:9:48.56 +1991051850,43083993845324032,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.089,1958-10-03,false,651726801.32,9:17:31.13 +-1245746904,64712828344164520,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.089,1975-08-23,true,991549329.203,21:16:29.55 +-1728788373,72270613091425760,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.089,1981-10-03,true,1216910479.56,2:15:45.10 +2029450647,40204877584077856,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.089,2000-03-19,false,846914216.268,2:10:17.54 +623001751,88815410197758304,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.090,1993-10-09,false,1087808156.62,2:47:37.34 +-1196922093,79532630047240832,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.090,1965-09-16,false,72385927.9493,22:38:19.10 +1038863571,67789720190000448,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.090,2012-04-08,false,247596590.341,13:58:14.26 +673792998,71348187901252280,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.090,1990-07-24,false,1337949586.13,2:55:52.25 +-1996969317,91714322716508944,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.090,2004-05-25,true,305266361.966,18:53:44.20 +-204902848,32801212158203824,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.090,1981-03-17,true,808753140.975,9:45:55.6 +-783197998,8474333755586448,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.090,2009-10-25,false,1022289929.06,17:46:15.31 +125433487,79041592800211424,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.091,1984-03-03,true,838176686.723,16:47:48.39 +-193493206,45807611367138824,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.091,1969-04-11,true,205522302.613,4:58:47.15 +-1758060797,26544676883004940,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.091,1998-03-19,false,47674740.8082,9:23:39.27 +1128497263,62043281079919144,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.091,1950-08-15,false,1253853755.88,22:28:10.7 +-1607451185,39031198465516880,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.091,1987-01-16,false,286366351.463,6:8:16.15 +798169838,16259555943555010,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.092,2012-04-20,false,574963346.592,16:19:43.29 +-450982404,21128006819242732,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.092,1964-09-13,true,1404918005.19,4:7:7.39 +2086623573,10514975097346694,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.092,1992-08-25,false,1016472907.68,5:46:26.14 +392953190,24593494006183208,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.092,1973-07-07,false,346228045.016,19:54:1.33 +1815739978,79740507594703168,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.092,1976-03-06,true,279663063.29,18:25:15.26 +1493868086,8535883830838006,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.092,1963-10-03,true,1186678663.13,1:47:32.51 +755322874,16250997357998018,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.093,1964-10-27,true,372410516.653,18:47:24.22 +1038188795,69615762994749624,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.093,1972-10-13,true,1300706422.1,2:25:33.54 +52830066,35439550132363448,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.093,1999-06-17,false,1402463866.75,17:18:27.17 +-1104900740,27118463632948664,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.093,1993-04-24,true,351516207.013,1:27:24.38 +-496895974,5750792305469430,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.093,1995-06-01,false,1118380037.79,3:20:17.30 +-1284745451,56772971340285176,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.093,1995-02-16,false,247689342.64,5:53:18.12 +879687671,66219397388995208,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.093,1977-06-26,false,268964760.817,10:7:37.27 +-2088880787,66303757050522632,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.093,1959-05-02,true,1305212373.59,18:57:40.32 +1139341989,22924527038262752,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.094,1993-06-19,false,1306579787.83,10:50:26.19 +1622704672,24992737786907872,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.094,1956-11-06,true,491363782.884,17:48:4.47 +-851417471,31675905357682588,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.094,1999-08-24,false,1286219407.95,16:29:32.13 +-1942897337,56076210579266616,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.094,1987-09-08,true,1059705053.89,3:4:36.56 +-1740308254,88808440928498608,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.094,1953-01-20,true,1110372081.03,16:43:20.11 +1099047295,89214482485062400,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.094,1950-04-13,false,767348255.081,21:58:16.41 +928570581,40952381967055104,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.094,1967-07-23,true,335662520.771,19:1:54.31 +-882915546,59465919677665392,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.094,1980-03-09,true,786046191.69,1:18:42.14 +2120167403,6319557874168638,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.094,2005-03-25,false,144754191.83,11:17:29.4 +1050800025,67263367339364128,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.095,1959-02-07,true,1432697025.93,19:44:45.21 +-1201504786,72237112673685248,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.095,1956-03-15,false,285256607.082,21:57:23.9 +759706141,9957729023830734,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.095,1992-01-05,false,280562176.915,2:41:20.2 +-273295760,20964497455515660,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.095,1996-08-20,false,1413200224.19,13:31:35.19 +-1820248027,82330715328224064,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.095,1984-11-14,false,880158479.27,21:34:15.32 +307049059,12920643815222846,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.095,1976-02-07,true,269973786.024,9:21:25.55 +-393203873,90933637920375008,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.095,1962-05-04,false,530700614.742,7:43:38.27 +-410286268,19300423724986636,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.096,1955-08-12,true,402927108.406,5:13:25.18 +-1798452777,36177090292793568,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.096,1985-08-04,true,259752123.304,16:53:45.1 +-1667872522,51073436351177288,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.096,1979-11-20,true,1430759218.69,10:33:39.40 +897320196,79765192248847584,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.096,1988-02-04,true,939581311.909,14:24:41.42 +683124347,55922348870061936,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.096,1988-10-09,true,124704419.663,19:9:38.17 +41691777,41199232334137120,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.096,1996-10-11,false,74160173.3174,20:28:33.34 +-808741838,59145662333103504,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.096,2002-05-09,false,528697382.763,22:22:43.50 +1650862963,3785556638526617,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.097,1968-05-01,true,633843747.899,20:38:39.53 +-1892303018,2072184156818104,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.097,1999-02-15,false,681009608.284,17:8:54.22 +-808444782,748148194580736,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.097,1984-06-05,true,984199210.958,3:46:32.35 +1069548288,5726708920157727,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.097,1952-10-18,false,581449747.073,18:58:4.4 +783581367,14193940809300388,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.097,1976-03-12,false,759601814.597,20:28:15.58 +185594643,65750953899159448,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.097,1995-01-18,true,1212957827.77,4:18:4.50 +444725901,74355329458447408,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.097,2008-10-19,false,912409092.371,10:22:3.29 +1652219377,73561463906546912,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.097,1981-04-04,false,848049732.904,7:36:6.18 +1996789732,69292241256829448,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.098,1955-05-15,false,445369323.396,22:38:34.37 +-1235132911,53755449542615688,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.098,1960-01-16,false,348948381.375,18:42:27.10 +746900487,67939549552462904,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.098,1977-04-15,true,1182941635.37,14:35:40.36 +7192682,82731699053658032,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.098,1961-04-09,false,446434274.796,15:57:35.28 +-1788955080,22208676518927340,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.098,1995-11-12,false,968002873.12,6:27:10.31 +1486129421,37535557825945552,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.098,1951-11-09,true,294332586.22,12:19:38.18 +-946412736,58591031758939200,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.098,1999-05-23,true,458117259.094,20:28:34.5 +-1987421332,84427378701573200,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.099,1968-09-26,true,566568923.75,8:46:41.45 +139005617,33405180728145500,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.099,1993-11-11,true,1092360798.56,3:52:24.3 +-1628202491,72661448782557472,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.099,1955-09-10,false,887766599.559,20:40:39.56 +551635743,79632341401913168,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.099,1967-03-18,true,388138422.91,6:16:18.13 +-279738646,42490753688518896,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.099,2013-03-01,true,210953508.902,13:54:21.29 +-450990616,35001874182578248,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.099,2012-05-08,true,656314813.739,2:36:39.35 +-505434156,4028510257009572,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.099,1954-10-21,true,1080201877.58,18:39:18.24 +1091112744,56861106830364632,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.099,1950-02-13,false,863695078.657,3:58:21.53 +411743510,82664824874522352,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.100,2005-06-25,false,57640383.6943,14:47:24.51 +-950596931,52863798031630712,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.100,2006-05-25,false,406889999.374,15:58:11.1 +964656929,32395496871167692,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.100,2012-06-22,true,153419837.547,16:51:51.30 +-1214663772,7471148899615898,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.100,1953-06-23,true,118656208.491,3:6:18.9 +-347230228,63476362551511848,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.100,2009-10-15,true,1301456971.36,13:38:37.15 +-1266071086,41944491347806776,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.100,1958-01-16,true,1356864038.78,18:43:32.10 +-76365095,24487659660014212,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.100,1989-05-21,true,574769169.775,9:23:58.51 +-1951601378,8261841513621555,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.101,1967-05-09,false,248509142.103,9:9:16.26 +-1662490526,42505609224525096,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.101,1996-10-17,false,255582064.667,3:40:20.48 +-527980614,25101201286965556,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.101,1978-05-09,false,81724235.4067,4:50:40.50 +-1574397803,22446695706915268,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.101,1959-11-27,true,1162326494.81,16:32:6.13 +-928992077,86422987322144192,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.101,1975-09-14,true,242903798.576,16:17:2.24 +2051740022,79697205789274128,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.101,1962-01-07,true,325972401.289,6:19:2.20 +1535250416,81393502126717264,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.101,1965-09-12,false,37164131.9211,20:19:28.58 +1723627190,67031925454816000,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.101,1960-08-26,true,1257483035.94,2:38:3.47 +125665284,4720039512535931,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.102,1992-08-12,false,1034783169.49,3:55:51.53 +865216632,33928929239176324,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.102,1954-06-11,true,631174330.759,11:8:19.24 +-499997248,34804974242173852,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.102,2009-02-27,true,1296176434.86,14:4:5.29 +220979680,36281839307260696,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.102,1979-02-15,true,721157330.029,10:32:45.13 +-138974446,29203957097993628,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.102,2009-01-23,false,1233622284.79,13:1:30.40 +-244420675,9138048326207202,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.102,1989-08-15,true,724132140.709,19:54:44.55 +366858932,89055664142046704,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.102,1981-09-21,false,425093115.328,18:31:19.25 +-1410387408,69442534989532384,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.102,2009-07-27,false,413300676.757,16:2:20.8 +777311768,84451146855518688,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.103,1989-02-16,false,513449398.018,18:32:48.30 +378277378,29738601898079736,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.103,1966-05-22,false,260083107.54,12:41:22.37 +1469207327,14620884529695458,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.103,1998-09-10,false,1186980615.59,10:48:36.15 +-1027496804,51694199543142336,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.103,1967-03-03,false,1010260178.16,17:47:38.31 +-657221803,45690158253281088,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.103,1999-03-27,false,397850641.755,16:9:40.17 +1484631994,84455140798890352,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.103,1981-04-20,true,244600940.839,15:45:51.53 +-362690841,31443035665142804,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.103,1958-03-05,false,951268971.899,6:47:4.32 +-1198585520,10650299063438326,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.104,1982-04-10,false,726835589.113,15:33:14.18 +-843054283,48736872569480872,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.104,1959-05-11,false,518399739.268,20:40:37.43 +-1194873070,52502789645973512,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.104,1953-07-10,false,548928671.341,2:43:34.22 +-1964277161,17495797095577458,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.104,1969-08-01,false,790266823.185,18:55:22.42 +-218553738,86401667263333088,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.104,1955-04-21,false,801873.890419,16:43:14.45 +-152716819,9868104422774856,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.104,1961-04-09,false,1178342229.65,18:20:39.42 +625368313,20282739472526828,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.104,1983-02-05,true,676595432.51,14:19:15.5 +1096081883,65672580771627960,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.104,1951-11-12,false,706783892.166,7:37:3.3 +-151128075,4183536030300846,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.105,2008-03-14,false,958663977.15,13:24:18.57 +1668181752,73602181284393056,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.105,2000-03-03,true,518040374.965,7:5:8.44 +907163382,13923557847811790,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.105,1967-06-19,false,917460335.918,6:40:26.32 +-740347187,33142031312689696,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.105,1976-06-27,true,501308451.925,7:58:34.43 +-676967753,30828721474311744,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.105,2003-06-18,false,1128705003.57,3:43:2.39 +-219223019,45188957043398032,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.105,1979-09-21,true,706029926.351,16:56:24.38 +-1035748861,61142494535113296,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.105,1968-06-14,true,1127859690.03,12:43:33.3 +657812407,45315442122132768,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.106,2011-03-13,true,8843969.34352,3:32:9.19 +971431921,70869124377349248,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.106,1980-09-04,true,487459420.416,1:10:8.47 +1341658683,10849527988014050,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.106,1971-07-06,false,424287498.439,14:14:14.48 +1908186009,49734797894735368,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.106,1972-02-27,true,1098442065.93,3:36:42.11 +1437326274,88986253445931264,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.106,1979-05-13,false,129251222.39,7:52:14.35 +356862158,2881778234055147,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.106,2004-01-06,true,94948992.0959,3:23:1.50 +-544456446,65473473512226472,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.106,1987-07-20,true,342143609.136,21:7:34.3 +2010281191,16214958434188596,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.107,1965-11-05,true,1186476163.11,10:55:52.42 +1754562237,19573539070512292,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.107,2012-10-26,false,1223589778.25,1:45:58.11 +-1056614069,23584661462361160,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.107,1971-01-19,true,407993864.868,1:37:29.6 +962820384,24713240876906528,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.107,1968-03-26,false,204431887.909,17:36:47.42 +33765593,50687253403742304,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.107,1984-04-03,true,1245778305.13,15:7:18.53 +852978332,63116576175117240,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.107,2013-02-15,false,468738020.052,15:40:58.34 +-1311617541,54569300981227416,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.107,1991-02-23,false,764794328.79,21:39:17.8 +-1577498067,61726958691729744,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.108,1963-09-17,false,285129974.704,14:3:45.1 +1305599709,39209955042844992,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.108,1994-10-02,true,807291036.627,10:36:32.53 +-942183916,11102774641608714,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.108,1968-10-22,false,125005790.175,18:13:47.1 +-1547464248,27100517444414364,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.108,1988-11-10,false,479911781.366,4:5:58.58 +475896431,9825487477275412,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.108,1980-09-06,true,406325695.559,22:35:13.41 +-2009145029,75804766832101728,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.108,2011-05-24,false,263409748.038,12:36:35.4 +-281352353,85112902653718176,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.108,1986-03-20,true,693135333.005,6:58:55.7 +-1991881263,71154071680452232,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.109,2003-02-08,false,1032480485.76,16:6:49.13 +1647623719,29283287521118608,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.109,1955-07-03,false,318677913.518,22:6:37.22 +754240822,66854496257760368,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.109,1970-05-12,false,376107061.306,8:36:24.26 +-630332972,51031734762439816,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.109,1976-03-25,false,1340002262.31,15:51:41.18 +-1539202593,60847730508276648,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.109,1957-07-16,true,206685481.589,5:22:36.53 +1348330126,52988479625144360,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.109,1965-01-26,true,1368686490.69,1:52:6.53 +1013291210,36010016493480716,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.109,1965-09-10,false,1296469124.97,4:47:22.1 +226452521,37974552016597880,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.110,2006-05-02,true,147766295.99,18:37:3.23 +1816095671,27508689844156920,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.110,1989-10-16,true,858043914.449,2:10:53.37 +-942499461,7252821698290698,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.110,2010-05-05,false,679212692.809,20:15:9.31 +-470774998,75885788641096128,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.110,1961-03-11,false,513061174.57,14:26:8.23 +-1219695935,10395011247175824,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.110,1965-03-18,false,317269175.365,2:58:47.23 +-1663071276,12517103574738974,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.110,1990-08-23,true,1112931856.23,7:22:20.14 +287353888,88694080314931264,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.110,1996-07-14,false,1409916330.32,11:13:54.42 +-1690799231,85499828292761040,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.111,1957-06-08,true,588239900.937,4:42:37.17 +-1415489908,6355995820799130,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.111,2003-11-04,false,634605748.359,6:19:37.34 +-241112348,25699792367726212,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.111,2001-10-03,true,740578424.881,6:8:37.36 +-107042233,24759927386062900,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.111,1988-10-03,false,220424726.968,17:8:29.29 +1132932250,9373106753841122,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.111,1965-06-08,true,288021427.7,7:1:39.23 +1642804408,2654454244227973,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.111,2008-07-16,true,113633104.515,20:52:36.45 +-220709061,74337204319487136,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.111,1980-11-13,false,323487985.044,15:42:3.17 +-1421614718,14841215036341054,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.112,1997-04-13,false,387427170.915,19:10:35.19 +36538792,23014459284229336,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.112,1986-04-18,true,886904313.03,13:38:42.7 +2127117961,60821811056974592,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.112,1958-10-02,false,704014378.675,16:7:13.5 +1697629600,66977989000578472,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.112,1973-05-14,false,517334146.683,13:38:48.56 +-1564205834,38588897140953744,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.112,2013-05-02,true,907142988.542,3:24:46.33 +1467849725,89141336953485856,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.112,1992-01-15,true,53475246.8489,6:25:5.51 +-1179463674,26246326588791244,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.112,1992-01-21,false,821300483.656,19:41:20.2 +1612377698,59600641356161264,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.112,1958-10-22,false,986883117.861,14:10:4.56 +2043347674,22282540967793460,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.113,1965-11-22,false,110339672.505,17:3:46.40 +-731138214,19867288858715220,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.113,1994-04-17,false,1083385531.89,6:18:9.52 +53868093,17601330859627480,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.113,1976-06-22,false,827694319.49,2:19:40.48 +2119648759,87043752749953184,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.113,1956-01-01,false,698641690.84,4:22:42.11 +1188148792,69620625513814112,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.113,1966-01-11,true,1174931028.98,18:34:45.26 +-375192187,12057121254810420,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.113,1952-08-02,false,834395577.101,14:41:35.27 +1834093556,80331206466289824,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.113,1998-10-15,true,859030001.853,13:20:13.10 +495894314,12675533550591570,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.114,1963-03-23,true,86245069.1337,19:2:7.37 +1746343747,54128867688896776,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.114,2013-02-08,false,435233899.494,7:14:23.7 +2007403606,6892502265780112,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.114,1968-04-06,false,1291079505.42,19:53:31.39 +1398929436,87923451670263792,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.114,1984-08-20,true,701623045.046,15:58:34.42 +604163002,69447478539989360,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.114,1959-03-24,true,465893378.162,9:30:3.45 +718435787,59530622349829760,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.114,1975-05-16,false,688644554.413,3:17:43.10 +-2140928718,23762955394511156,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.114,1968-03-02,true,1352668489.91,19:24:6.44 +-1819059217,71301740077112616,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.115,1958-03-17,true,997707721.023,8:16:5.38 +2014976406,63363279828955464,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.115,2011-05-24,true,1089568342.19,8:9:28.48 +-1145341706,73482376930410272,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.115,1990-05-03,true,402329818.002,15:33:5.5 +-943269626,21728984359973460,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.115,1974-03-02,false,1043981602.46,14:7:17.25 +-236211431,24753788694420216,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.115,1950-09-02,false,1211127482.36,13:46:33.3 +-1777460516,22471243439054684,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.115,1993-01-09,true,460723847.904,4:49:25.11 +-849042748,35372462605936160,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.115,1955-06-09,true,667837006.75,22:50:41.6 +1246547804,44499290633768656,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.116,1985-10-06,true,357302022.981,7:52:21.30 +1124988445,3941520941545165,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.116,2013-07-26,false,43606558.7911,10:10:12.39 +-1039782114,54293272624691896,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.116,1990-11-22,false,1114988834.6,4:44:37.55 +-1653773793,10707449370783078,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.116,1961-08-19,false,159351028.346,12:53:34.56 +-2031836842,24260878583795488,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.116,1971-06-15,true,245627608.979,8:23:49.40 +1470376995,40692075232865360,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.116,1991-01-23,false,758478408.232,22:5:32.44 +392940148,57101486212013560,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.116,1954-02-07,false,1164514326.23,19:52:52.51 +1439974118,52051844738645312,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.116,2008-02-23,false,436983418.269,15:39:17.49 +1961237317,52034450024138848,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.117,1999-04-11,false,373787230.441,4:14:14.5 +1359756006,14654692064652698,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.117,1992-01-08,true,159994582.645,3:41:51.20 +1817014382,39210708766405568,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.117,1967-05-25,true,241103452.227,17:2:42.39 +1719622329,76940410386222496,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.117,1968-07-18,false,312428424.098,19:15:6.26 +-717322175,33015863629429044,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.117,1971-06-04,false,1378712766.89,4:19:52.22 +-400772079,1601318378826567,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.117,1959-09-18,false,338013419.767,16:41:10.58 +-1038880883,61627182218027040,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.117,1978-07-11,false,849722622.136,12:27:40.39 +-1588395191,74826472058613312,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.118,2004-07-13,false,631264627.862,8:31:11.34 +1126842195,90684663782556528,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.118,1954-07-18,true,319560745.98,15:54:35.7 +1617036814,972456845741332,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.118,2010-11-05,false,52875154.4514,22:16:20.50 +-301898459,4495369541015419,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.118,1962-02-19,false,860230174.215,1:8:41.18 +-1753752543,24646349077829788,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.118,1990-11-18,true,130656010.611,7:50:19.3 +-876656749,38571125641289800,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.118,1961-10-16,false,136784437.454,7:1:33.22 +-657297893,48097700597520672,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.118,1968-05-27,false,1235718015.85,5:17:43.20 +2037163091,45920076370029376,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.118,2004-05-22,false,581977696.438,5:51:52.8 +-535261121,4590843133929749,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.119,1962-03-16,false,544688497.231,19:19:46.32 +673564119,7977428474472069,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.119,1975-06-25,false,675517688.811,5:57:1.56 +-825996492,13168664735152394,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.119,1979-09-23,true,720546283.124,15:49:1.11 +1990201010,59135514889044032,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.119,1986-03-01,false,595304863.389,10:31:29.39 +1925035772,64768129074563480,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.119,1987-07-17,false,1183167003.66,3:26:23.54 +961445101,68251999280964752,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.119,1962-11-01,true,832236477.969,15:44:40.14 +346622878,64509431902397176,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.119,1968-08-05,false,734929290.986,12:50:12.56 +1470251242,48833355217297296,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.120,2002-07-24,true,683104171.704,5:44:56.45 +1896203636,45711741166695752,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.120,1958-02-02,true,965513924.065,15:39:26.33 +492709200,54868286339009376,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.120,1994-05-06,false,962906994.471,5:49:37.6 +70621881,17130054813348700,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.120,1976-05-25,true,904692016.605,7:39:51.31 +1047702045,87481476572497600,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.120,1960-11-02,true,362466966.767,14:21:7.9 +1973347523,87384954743041552,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.120,1960-09-08,true,1141609891.03,10:39:5.19 +1677208959,65193658537944168,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.120,1963-01-14,false,150785702.848,11:42:12.34 +-1242961405,22092137447108884,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.121,1953-06-18,false,1185681289.09,8:22:10.17 +1740961412,91146747355385856,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.121,2006-04-01,true,429654803.52,7:45:22.18 +-1918487491,54577310342033416,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.121,1990-01-21,true,1043148519.04,3:38:31.55 +-335541284,54038242932037104,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.121,2008-02-01,true,1429207438.11,15:4:50.19 +-237210540,50204264960417904,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.121,2009-07-16,true,471553089.888,13:14:22.31 +1544903854,33637455574750732,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.121,1963-05-06,false,852422202.925,2:36:40.57 +1430446042,40706745620595416,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.121,1989-09-20,false,1206248821.72,17:57:16.41 +-1203201935,17476765730468618,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.122,1951-05-08,true,632178482.341,2:12:49.38 +-961717327,2424909966269737,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.122,1992-08-26,false,63069731.4277,5:48:1.26 +-992459566,2251975638602383,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.122,1960-06-05,true,20418087.8119,21:52:36.4 +-1419839913,71788461719557816,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.122,2010-10-21,false,1158828554.48,10:27:33.11 +-1011804285,58113171485829912,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.122,1965-08-18,false,1397415117.88,4:28:36.53 +-1252617870,41362836346013648,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.122,1995-04-03,false,1265667268.35,4:55:9.6 +-2093341145,26950389805160028,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.122,2006-11-03,false,715862719.329,18:50:21.52 +-2104619524,37976369970342720,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.122,1974-05-17,true,175894952.673,14:10:26.27 +823619956,14564965931745904,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.123,1967-08-08,false,1192606374.07,18:47:29.35 +-1404418197,28725341528141560,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.123,1986-05-06,false,278241216.829,17:43:39.31 +1203746388,56838214851719088,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.123,1951-01-21,true,76718115.4354,9:52:50.13 +-554059261,72951359917073824,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.123,1996-11-04,false,911165474.494,18:47:49.50 +149840547,6395621263015844,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.123,1990-05-15,false,1062002206.68,14:40:26.22 +-81975352,90206073139760880,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.123,1978-05-06,false,46308743.376,19:37:49.7 +-223863853,33145685227947696,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.123,2007-10-10,true,191132326.355,20:16:54.34 +345173924,44774626959030448,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.124,1969-02-18,false,514488435.809,7:14:33.38 +2016555398,68813571172366104,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.124,2010-03-20,false,1053420577.31,2:9:37.34 +779396338,81398009312524944,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.124,1990-09-14,false,86067161.9137,8:19:10.40 +-1496792295,61952627586089408,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.124,1962-09-09,true,343135658.133,10:25:14.2 +1298833782,14485645355059928,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.124,2008-10-17,true,1311195021.72,5:26:17.51 +-2036727558,88976996595759520,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.124,2002-04-10,true,597349795.291,19:37:31.33 +-49892115,44916145870813768,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.124,1998-10-07,true,89979216.0561,12:22:25.40 +2081474720,11397862712919582,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.125,1980-09-17,true,152296481.988,8:20:57.28 +-1802219364,23329025941454336,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.125,1959-04-20,false,866635020.324,15:13:45.27 +-1464676525,66560790636896664,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.125,2008-01-24,true,237920664.977,3:56:23.27 +-891610634,41331380378154176,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.125,1983-09-14,false,1104408923.17,13:38:29.2 +-949405919,65026720075495520,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.125,1992-05-20,false,1216007563.45,12:30:26.57 +-154119912,29410569317768604,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.125,1954-03-17,true,1098839371.79,16:21:45.51 +-1103274647,42705043139540088,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.125,2001-01-03,true,1196278529.94,10:5:51.28 +-1286420645,71805368403452704,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.126,1971-01-26,true,1411405531.57,7:55:36.14 +-896436705,84354145243974224,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.126,1991-05-01,true,1031349791.01,16:16:48.14 +144510107,22865141017229036,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.126,1988-07-12,true,1393993722.14,18:52:44.48 +-968626612,90089189428320048,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.126,1957-01-18,true,710212220.204,13:54:46.53 +779766830,25165188133486296,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.126,1987-05-07,false,1022594543.89,20:44:52.47 +885985912,85983759974773184,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.126,1977-11-15,false,198728565.772,16:14:58.35 +1403344140,12290131588054640,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.126,2009-04-17,true,286554855.593,4:12:56.43 +-1375002870,67149411183408320,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.127,1997-07-23,true,726557236.686,8:58:34.13 +1394729502,73714712643257296,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.127,1981-11-02,true,1420346302.47,6:56:48.40 +-927067709,53735570917700600,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.127,1957-10-27,false,1353498482.28,3:54:36.38 +1824596235,18374384449866300,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.127,1964-03-21,true,1135150382.95,5:40:37.22 +-942502142,32176009404208548,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.127,1976-04-26,false,516774065.063,11:20:11.18 +805214051,27270423642210408,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.127,1951-08-20,false,653827237.529,14:51:28.17 +964001049,10070483599979684,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.128,1965-08-25,false,80283323.1999,20:29:14.16 +-14396296,81953214955416688,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.128,1960-01-05,false,1092499224.11,18:24:41.19 +-2114468992,86387138899732640,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.128,2012-11-06,true,929433335.828,22:9:56.23 +1151666341,30794105535469232,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.128,1966-01-10,true,410172002.892,10:58:39.35 +2145613587,60154682804864112,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.128,1985-05-09,false,1014007840.91,8:52:23.24 +865223204,20180897082051040,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.128,1972-11-22,false,138749689.933,4:7:13.35 +-537127226,72070190342653920,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.128,1952-09-19,true,865479535.869,3:39:35.54 +-1846917767,17020948699783486,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.128,1999-08-08,true,134603782.513,3:7:28.56 +-374954979,13858609363444132,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.129,1969-08-14,false,668583234.75,3:44:26.15 +-914856186,53945106462158072,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.129,1951-01-20,true,949310610.663,14:6:35.43 +426830473,39016626339139264,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.129,1952-06-11,true,1043290509.14,3:5:22.17 +1286980604,70259262522637416,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.129,2008-03-17,true,253620357.268,14:57:1.23 +-863910036,75983265497258528,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.129,2013-03-13,true,639236590.379,4:22:31.4 +-1618017684,38357837501655984,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.129,1993-02-07,false,1320796916.69,8:41:37.51 +1441162935,25494601722605908,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.129,1984-08-08,false,287434156.301,1:45:2.43 +-748958797,49424730407944760,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.129,1974-03-13,false,467256370.192,18:44:23.30 +1200093803,26377999230933104,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.130,1962-07-12,true,53494311.7469,19:45:44.34 +-2132794615,46370906254268072,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.130,1956-07-01,true,950153324.769,3:5:12.49 +-1943757503,41453690149310056,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.130,1968-08-15,false,1184988203.16,12:11:5.44 +1127360348,38451438427396968,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.130,1953-02-08,false,267370064.462,15:28:45.54 +192945911,11132913356725074,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.130,1980-03-03,true,1330093179.34,1:30:8.9 +309920587,8674788901254779,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.130,1953-03-02,false,1114023480.9,10:15:45.13 +-969676974,67099350697349640,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.130,1993-05-27,true,236165242.791,21:38:49.11 +21294776,66317786579764624,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.131,1962-07-12,false,871124298.653,3:4:13.24 +-1904248921,5973105166530519,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.131,1992-04-08,true,679168907.751,20:15:3.45 +-953739749,62770129441313976,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.131,1984-11-06,true,606302610.488,22:17:40.50 +-1782058678,90212496178744816,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.131,1973-08-12,true,884901593.434,12:48:55.19 +-1836592982,77728470616096752,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.131,1967-05-17,true,880523433.158,8:6:51.11 +-1240982064,75815498887311312,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.131,2011-08-27,true,447457974.964,10:16:51.41 +-1878247703,59529855549354864,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.131,1976-07-07,false,1359508864.26,16:2:28.4 +-948989994,67096616294878432,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.132,1960-06-27,false,1207648692.15,20:43:26.45 +1904169770,37869550647448320,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.132,1977-02-15,false,514532044.28,8:7:18.18 +978631441,75340470120348848,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.132,1995-06-26,true,1098196624.24,17:52:11.2 +891684116,8098208779255409,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.132,1993-09-17,true,260370369.03,1:49:50.17 +291053026,27382721682790964,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.132,1978-06-10,true,1297030210.95,21:4:54.50 +1622633150,27568693006809056,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.132,1973-01-05,false,671719839.438,6:32:17.6 +-2008766091,57415954541105120,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.132,1982-04-03,true,909003249.863,15:9:29.18 +207505948,58622778239589080,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.133,1978-08-26,false,1262543014.35,8:47:50.53 +166004666,43598298482263104,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.133,2009-08-03,true,962922373.892,18:17:55.20 +-1030176237,17034239459409900,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.133,1967-09-24,true,936488172.277,19:29:22.39 +1487546366,53958261916147464,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.133,2002-10-18,false,122585153.218,3:3:27.25 +-217821082,977048730253465,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.133,1974-08-12,true,855625508.71,21:47:56.4 +-285473687,60380491935066632,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.133,1992-04-14,false,878628243.024,14:19:20.43 +263495999,8845240841796229,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.133,1966-04-12,true,647334519.534,4:16:34.31 +1588749110,41059419762318768,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.134,2000-06-09,false,595372186.19,11:57:39.1 +1407940400,4816653297304115,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.134,1982-10-03,false,705303007.643,19:48:9.39 +1185343150,39471836367859952,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.134,1986-05-04,true,839221554.762,1:45:48.5 +1876892860,7230280349617603,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.134,1957-01-23,false,1271674606.28,12:36:38.19 +1038921366,6733513041464679,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.134,1995-01-17,true,524278772.689,6:54:3.29 +383302230,57981694947442096,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.134,1958-04-22,false,686998726.85,22:4:19.13 +-1591100444,17714940819800668,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.134,1972-05-18,false,327573162.734,14:14:9.2 +-2052750379,11614881876762204,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.135,2000-10-08,true,29397727.58,7:18:7.42 +1316541622,60427379436495752,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.135,1968-07-20,false,94592936.7916,20:40:24.9 +265335063,52794414837438560,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.135,1983-04-27,false,1058583784.01,6:1:45.57 +1679112885,66132083361125344,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.135,1999-05-04,false,1380822038.4,5:18:41.35 +-251721706,63671191867207672,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.135,1974-06-27,true,647906643.037,15:17:23.11 +1482001497,84311533557654240,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.135,1953-05-09,true,954223685.561,5:20:52.17 +-1872964425,15564321786918606,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.135,1957-04-04,false,487263791.387,20:20:51.9 +-1936456323,19382323636853668,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.135,2002-08-10,false,114255376.633,12:42:27.48 +-1185677809,64395024613567856,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.136,1953-01-26,true,1342834170.75,7:53:16.19 +384159115,1787952882689515,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.136,1965-07-22,false,1233026241.28,11:38:21.8 +389584936,23816383400416276,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.136,1964-03-09,true,756482818.22,18:43:46.23 +-2109716354,42365399861991264,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.136,2001-04-26,false,851442536.657,18:3:46.42 +1923172911,91727794115215680,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.136,2009-05-26,true,6646492.7809,9:11:22.20 +736487607,71912222092076832,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.136,1989-07-09,false,1049186634.75,6:25:45.2 +-2036610380,21920276771537696,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.137,1956-09-17,false,906817556.831,11:39:56.19 +-297395093,33801504333747548,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.137,1957-11-03,false,258016605.381,7:48:30.21 +-353171087,63017576660724264,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.137,1974-03-27,false,925652252.657,7:50:50.23 +471380006,23188604700148308,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.137,2007-06-19,false,851135997.016,17:51:41.4 +1700489756,29660009338164880,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.137,1974-01-03,false,1379510477.31,18:49:15.47 +2024454251,52583935987799584,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.137,1960-03-13,false,168015426.377,8:15:11.47 +1369161287,78362535920387712,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.137,1999-07-01,true,527242887.641,6:26:44.56 +-225235113,60011955413450280,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.137,1951-02-04,true,726177352.504,9:13:3.25 +-1303389354,7295843598058998,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.138,1994-07-23,false,821936011.32,17:20:22.38 +1908197643,51032096846964592,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.138,1957-07-20,false,1167624160.36,15:45:21.37 +1974791686,41031409974603472,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.138,1992-01-08,true,143762552.325,6:13:24.17 +-687593960,15904024311297414,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.138,2001-06-08,true,429499297.878,7:8:26.1 +-576118416,45753558180622288,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.138,1979-10-22,true,989001290.608,22:52:41.58 +197969808,59407622313050104,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.138,1984-07-11,false,1233060195.98,15:41:47.47 +1225396785,52439175566703136,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.138,1999-08-18,true,699438723.747,20:8:34.1 +-1211054298,44622768422268416,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.139,2010-04-20,true,325559271.39,8:4:31.36 +-1970984927,49297571825183336,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.139,1987-09-05,true,532483287.573,10:13:6.43 +-405745761,62721825727444360,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.139,1961-02-10,true,1078006636.97,22:51:44.31 +826949938,45379630438533408,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.139,1953-04-03,false,196139656.158,20:45:20.19 +-1879052428,84314542530145952,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.139,2001-01-26,true,825295879.266,19:4:57.23 +-1297646364,14345492288925010,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.139,2012-01-19,false,520590415.693,14:38:33.3 +1941241666,37975458331367632,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.139,2013-05-17,false,506525803.757,4:33:38.3 +219446612,48130158053389992,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.140,1950-11-03,false,737314151.212,3:28:38.56 +-821170479,85629214049701152,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.140,1999-04-01,false,1296747400.83,5:6:49.46 +1820035613,74973772356068592,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.140,1953-07-04,true,652698778.782,8:45:56.22 +-11807180,82935468194740160,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.140,2001-10-05,false,316574988.414,9:33:20.15 +-547721426,63712858854630312,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.140,1976-10-20,true,690267080.27,17:47:21.25 +-1506564888,61897590748110736,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.140,1982-05-19,false,312858240.833,17:21:29.11 +513237156,18303847131918696,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.140,1952-11-24,true,717521542.035,1:41:58.14 +1376410958,53337906390647208,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.141,1982-01-15,true,369638677.195,18:12:27.3 +-158570483,21668420661465704,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.141,1959-07-06,true,273355010.827,16:3:1.44 +-774760869,61330256842093016,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.141,1958-04-22,false,973941184.095,18:3:28.12 +-1849455391,17040875795333796,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.141,1969-02-18,true,374463146.855,20:8:43.49 +278122112,21375229828135108,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.141,1978-08-18,true,456879918.266,18:2:3.9 +488716370,59558882140569496,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.141,1973-09-13,true,1006654708.11,15:2:28.34 +-469218411,64075454144339136,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.141,1973-05-21,false,130652529.428,10:41:2.47 +1606860950,27010636401251880,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.142,1990-04-09,false,455634284.967,12:47:13.18 +-1340286628,23252215002756548,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.142,1951-05-27,false,211332879.576,5:24:11.12 +1645239530,28226097161143584,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.142,1981-08-13,true,812916051.361,4:35:10.27 +1609153191,92090612387230784,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.142,1996-09-16,false,209266192.384,15:24:44.1 +-888701199,25074443481965344,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.142,2004-06-26,false,45636268.1906,14:19:36.18 +1011602070,75398335909088848,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.142,1974-11-21,false,252175399.374,2:27:27.19 +762628869,28927954057221960,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.142,1998-01-17,true,166373522.269,11:43:13.46 +-192256160,78630381586045136,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.143,1969-01-09,false,1403070187.6,9:23:46.4 +-1119928090,11279788460513638,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.143,1951-07-14,true,589749408.142,10:3:26.26 +-215961441,55721229939181016,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.143,1972-04-09,false,699549564.789,13:35:5.10 +-2131145645,14434066490829866,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.143,1989-01-04,false,481021924.409,12:36:23.28 +864727191,64536315913303920,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.143,1977-07-12,true,1381715698.47,21:2:32.4 +1031031358,31966064913444220,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.143,2012-01-20,true,1217153290.32,15:37:42.23 +580920172,40235644152567360,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.143,1968-02-26,true,1171072084.62,15:3:17.14 +2103211339,31412700161038236,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.143,1991-02-02,false,83791433.7501,17:47:18.52 +1150335871,82333186651858672,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.144,1961-09-21,false,111397362.847,2:45:23.20 +-1961388756,75037655251344832,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.144,1987-07-10,false,176825944.676,1:56:7.57 +1164184916,32400745217003724,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.144,1978-03-21,true,1171136288.19,15:19:4.17 +1367590102,78037707020726944,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.144,1994-01-12,false,307863038.515,8:39:17.36 +626737214,79745181963728848,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.144,1991-07-25,false,641086437.552,21:38:43.58 +-834777580,15430538640732314,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.144,1986-07-23,false,1183093434.44,9:51:36.6 +769211373,77797536042824464,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.144,1998-11-18,false,787093716.289,4:11:18.14 +25711523,54563069366460304,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.145,1988-07-09,true,1265362490.41,20:35:49.5 +-528117940,4007844755678781,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.145,1992-01-16,true,201990816.277,6:8:1.15 +-635133992,31290342772175144,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.145,1959-06-10,true,1009627990.87,19:31:36.11 +439666179,6204416353864868,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.145,1969-11-11,false,775754902.134,7:27:45.26 +-1241730991,84749863205429552,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.145,1994-07-21,true,63999421.5652,17:25:22.14 +-781002717,41125980333680856,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.145,2000-08-25,false,254283008.645,8:49:3.15 +1202321253,24547646991194952,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.145,2011-04-08,true,901865444.229,11:6:53.31 +1573634218,14907602264368230,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.146,2003-07-13,false,1164752232.53,15:39:1.20 +132769368,57956636292280680,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.146,1995-07-21,false,970555753.478,11:20:26.56 +-645228040,59334520523668472,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.146,1996-04-01,true,106030783.815,12:19:26.48 +710245250,68685744017351960,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.146,1992-09-11,true,817517827.551,5:54:2.45 +1602460898,32362782188560332,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.146,1987-02-18,true,157067689.386,8:52:2.26 +-729366832,13938378433274410,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.146,1964-08-25,false,161223844.573,20:17:58.55 +1349856103,66872972721475832,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.146,2013-03-17,false,551559213.321,21:2:21.21 +-1909101811,5900025511045130,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.147,1988-02-20,true,50337773.3282,14:13:51.39 +218110934,58501197573092608,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.147,1968-01-08,true,759516211.229,4:43:42.13 +198155892,33339764308950948,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.147,1974-11-14,true,590400064.204,7:43:31.25 +2017073713,63366845200537216,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.147,1998-09-13,false,60455468.3244,3:25:41.11 +-586725057,88350468795364320,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.147,1999-01-22,true,771757602.321,3:14:19.57 +-699723528,44853004264136928,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.147,1966-02-05,true,972471523.54,13:29:5.46 +458365511,16537794520705904,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.148,1990-04-07,false,1090278747.52,22:50:54.39 +1546973398,27904860949044440,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.148,2004-08-07,false,484920326.22,16:31:29.13 +568053224,25042553146920336,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.148,1979-05-26,true,735256028.376,3:25:22.18 +-642351844,79853020322803904,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.148,1994-02-11,false,940253544.764,4:38:51.48 +1069545851,86397889086213760,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.148,1976-10-22,true,443683038.114,3:58:22.50 +-919851739,77306901718344480,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.148,1980-08-02,false,806448325.072,22:46:40.54 +-2050779519,52011537741904344,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.148,2003-09-01,false,374720115.685,17:34:18.10 +-1121619282,18306961321477040,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.148,1963-09-24,false,341042637.012,13:4:53.13 +-1074784325,74624022005653328,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.149,2009-09-23,false,305349472.819,8:12:52.47 +-1869533882,40257781249360752,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.149,1967-08-01,true,640817548.399,21:2:44.7 +-1556348773,18509848487626956,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.149,1972-05-09,true,575945631.263,15:30:44.40 +1121407185,39536944454009016,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.149,1956-05-02,false,444568880.237,10:53:34.31 +-77107476,8052626335962010,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.149,1992-02-13,false,931402058.967,9:4:4.10 +1999901037,55532728187861744,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.149,1995-11-03,false,1117784554.44,21:15:18.12 +-990458189,88109854996365216,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.149,2002-05-11,false,965461928.689,21:14:58.15 +796009438,77314891769629536,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.150,2010-05-01,true,440187650.425,16:6:33.27 +-322313422,19957944525573080,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.150,1955-01-04,false,18312783.5225,22:34:24.43 +236977326,47396483913716920,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.150,1964-01-21,false,271027205.606,22:1:13.34 +1710216340,84964706099053248,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.150,1950-04-16,true,403538565.051,13:9:10.46 +-579674671,84742310256625872,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.150,1973-03-18,true,464521903.262,3:17:13.8 +-763312598,25858712139279852,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.150,2001-01-19,false,1131437472.72,14:18:6.3 +1456500762,17415156708528046,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.150,1994-03-26,false,798896574.088,7:36:55.50 +-255418133,431452457890232,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.151,1976-03-02,false,62620414.2882,14:18:30.3 +978285864,29144549572444200,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.151,1983-04-13,true,739574790.446,15:21:30.14 +1560207331,79872612889431152,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.151,1959-11-02,true,27576046.0097,15:43:45.11 +1895908748,85427105017117392,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.151,1986-04-27,false,539271160.147,13:3:54.55 +-1375442204,79282746248633392,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.151,2003-08-21,false,69344530.8598,21:24:35.29 +535244803,34300251994059644,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.151,1967-02-16,false,278294637.257,18:21:43.45 +-739666778,15167027508585554,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.151,1953-01-26,false,40077324.9008,17:30:3.43 +719680863,35626471884719800,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.152,1976-08-03,false,39665138.5864,3:42:23.6 +-418118206,3446992359226900,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.152,1960-11-05,true,1158246587.5,11:49:16.9 +-2127862432,33094268985672264,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.152,1958-02-26,false,806158093.034,14:16:18.47 +-1300237207,34854560377311364,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.152,1975-04-22,false,1382129887.95,14:46:8.42 +736277873,33758768605345804,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.152,2009-01-14,true,1379115417.15,4:42:58.8 +-321774535,32342254112077200,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.152,1970-11-16,false,668684508.343,2:39:46.1 +-1924138781,35313251171289868,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.152,1994-10-18,false,668443728.046,18:13:50.6 +1358067380,7631856598599404,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.153,2011-10-06,true,323216902.904,18:3:41.19 +-1127285437,79399097835406832,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.153,1988-02-19,true,542821497.938,2:51:9.1 +21725200,91014456988029648,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.153,1957-01-21,true,943558981.736,14:38:58.34 +-459918628,71475778438312184,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.153,1979-05-24,true,510076373.299,15:32:11.51 +916980086,26050692398413436,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.153,1993-11-26,true,969552347.663,20:44:7.34 +-549993528,12204444304484946,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.153,1970-03-12,true,1005218988.97,3:44:17.35 +1349773451,8795501481565051,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.153,1989-09-13,false,877370309.475,1:25:1.33 +167400696,85737990668468336,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.153,1971-02-24,true,301458372.273,7:44:46.21 +-1240785118,89346607902051312,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.154,1973-02-21,false,1208073806.99,12:15:12.19 +-1825395594,90981761333893744,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.154,1953-05-08,true,513384386.726,10:34:34.42 +1274260784,54123029794176240,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.154,1971-03-19,false,281160789.0,14:35:7.57 +1448180109,43530462209899816,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.154,1976-09-16,true,652923706.957,13:41:33.47 +-1914248977,205204836308449,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.154,1971-08-11,false,1111521833.62,10:12:1.34 +-84002335,64718821320584656,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.154,2013-02-03,true,389709337.974,4:15:51.5 +-1469446984,55546602447687064,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.154,1960-11-16,false,1325580255.41,9:21:17.32 +551110190,37718747238981104,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.155,1966-10-24,true,930978406.866,18:58:2.13 +-552464943,20008653215510048,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.155,1962-08-27,false,1054131815.94,8:31:51.12 +1591721481,59880374848310752,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.155,2001-10-13,false,342645287.686,14:23:3.41 +-1721095982,67381063630332040,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.155,2000-10-21,true,1021500552.85,5:11:45.36 +-1019311338,32412293662200240,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.155,1951-07-07,false,581608080.684,19:56:42.17 +-1870733042,82644921354112016,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.155,1984-07-02,true,911808930.11,2:9:13.46 +-2106855892,88477158142970784,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.155,1992-03-18,false,81240014.0236,21:46:22.9 +-18568590,80508291960662128,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.156,2000-04-16,true,342576291.962,18:31:32.2 +731225467,48455601351155896,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.156,1960-06-23,true,914324208.94,12:56:42.19 +-1083100932,65322716811116584,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.156,1977-06-12,false,1376292754.17,5:38:29.46 +-1317117400,76470625982376096,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.156,1983-05-12,false,65616919.5783,6:56:9.13 +-167430911,79314372137746448,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.156,1999-06-13,true,450720211.569,9:18:10.12 +1929484201,61212377909520832,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.156,1999-01-21,false,781941212.446,7:53:14.14 +-1687744821,53333030975095544,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.156,2000-09-02,true,1173320736.5,17:39:38.11 +138792177,47311649494190520,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.157,1977-03-04,true,1047058543.91,4:16:25.27 +-106891600,45474421915232176,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.157,1995-10-11,true,1339864890.29,6:48:31.17 +-1534096935,17616839889754410,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.157,1958-06-15,false,1410257116.35,19:41:53.54 +-1988406922,6967885319508501,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.157,1952-05-18,true,117111930.927,2:2:57.10 +476728988,30564716345834568,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.157,1994-03-19,true,2631672.98098,6:35:55.37 +903922960,63693091602760336,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.157,1961-11-23,false,602177635.174,4:23:4.51 +-1120413979,54682959509161872,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.157,1980-03-14,false,509194710.999,11:14:39.54 +316504025,58582632517764848,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.158,1952-08-24,false,1079795966.74,11:14:29.43 +-269184936,50574201186815680,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.158,2000-01-14,false,815037022.755,22:40:57.4 +-671121809,43757827282188096,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.158,1959-03-21,true,1105497758.34,7:48:19.52 +-2117963004,74555954778819792,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.158,1957-06-27,false,999876882.095,17:21:37.41 +-1440420612,68481597068345072,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.158,1985-02-25,true,473125012.508,17:43:42.7 +71942186,46962189924010464,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.158,1983-05-05,false,310728127.907,18:18:54.41 +1972851641,32371211574744340,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.158,1999-03-07,true,978949376.736,10:12:57.44 +1522763217,87080536465506720,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.159,1984-09-03,false,573332423.734,22:49:56.17 +83518248,54088345007952272,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.159,1975-05-13,true,311325413.416,22:50:52.8 +-1527001441,47283732910168032,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.159,1971-11-14,true,931554184.349,17:35:40.57 +743933475,23831715833309808,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.159,1997-07-06,false,815352321.921,5:48:22.51 +1749703601,8085255194094377,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.159,1966-08-06,false,200141022.527,3:50:2.23 +-1930245713,7950433612783012,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.159,2006-10-25,true,759202479.409,12:9:11.46 +-114701023,25694772640143944,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.159,2010-05-10,false,1082255681.3,14:15:45.14 +-1730070436,45423503618097912,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.160,2000-11-25,true,1047975032.87,4:46:31.9 +1052328186,72305858074246560,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.160,1955-04-19,false,159915789.983,3:40:38.3 +1633501609,51902445747265200,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.160,2011-10-09,false,113313311.102,18:54:42.23 +2042495028,55573857671168880,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.160,2012-05-27,true,1027905183.92,13:12:7.38 +1911157904,61416665061268520,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.160,1998-07-09,false,1024150976.6,17:18:27.35 +1532186987,59157431766874336,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.160,2002-10-21,false,560344777.705,15:14:57.16 +-1475008287,45913536971426456,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.160,1955-02-04,false,2467277.34113,7:43:31.7 +-648491086,53359691251550176,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.161,1965-11-05,false,1015551817.77,9:14:46.11 +-472585990,85232073433348592,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.161,1997-02-23,true,50205384.8725,20:43:33.54 +-424354106,51673886066912440,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.161,2011-05-27,true,560762500.742,11:11:30.48 +375055028,47109867210424400,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.161,1963-06-13,false,1427234666.22,18:43:16.2 +-1773453949,12181600926602004,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.161,1978-03-22,true,1104899832.58,1:8:38.53 +1210007839,75722207467253792,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.161,1987-07-09,false,426811064.364,7:27:2.49 +52965893,22106873506608028,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.161,1964-03-04,true,1170542701.08,9:37:16.9 +-1364417144,15980649319365582,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.161,1996-05-25,false,70005450.0039,5:25:55.14 +-1420797238,66440605474895032,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.162,1994-01-05,false,673381605.28,10:41:48.57 +689739821,81249603673083056,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.162,1978-02-16,false,809701170.198,14:31:39.30 +843164439,36324827836980856,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.162,1994-03-03,true,82174989.2798,3:30:38.1 +-193085978,53268151090445992,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.162,1984-03-14,true,79020719.2068,1:53:55.50 +-1055806174,30930326830854872,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.162,1978-07-18,true,1177631590.46,14:29:44.16 +-1865045234,55591122015523128,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.162,1978-01-15,false,1050951300.47,11:13:28.25 +-883846209,51414565106391288,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.162,1972-04-11,false,196216397.032,16:43:57.43 +-1939020209,19171641201505176,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.163,1967-08-08,true,971052925.212,19:51:24.44 +1038972656,21697384521683448,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.163,1981-10-12,false,1249160348.84,12:24:51.33 +-123399913,20094373020742860,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.163,1955-06-14,true,1009054338.08,22:58:1.43 +18095844,52316855180991080,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.163,2005-05-22,true,795809285.373,10:7:42.21 +173694350,58412357969897128,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.163,2004-07-08,false,764721274.132,12:34:7.49 +-1884664282,15799236766976594,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.163,1988-02-14,false,649306679.09,4:28:55.1 +1553945026,89379727241023664,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.163,1954-02-24,true,167073480.674,9:5:38.55 +-1002731417,31069714337460100,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.164,1953-06-21,false,1185907927.59,9:37:7.3 +68045289,33022584466836204,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.164,2004-06-03,true,429701438.107,9:51:23.10 +548997325,895761848663040,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.164,1967-06-22,true,433467579.844,12:52:1.11 +-1835743347,47319312854641256,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.164,1974-09-13,true,888878960.365,20:1:52.3 +-775868122,65784046607016840,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.164,1969-07-10,true,672094487.936,4:24:54.23 +740315542,84069052876339984,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.164,2009-05-15,true,878929551.005,16:8:56.23 +252422452,27081007148768064,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.164,1986-03-23,true,80092228.5643,2:24:12.49 +219630875,83277479569614976,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.165,1972-07-15,true,207473185.534,13:27:38.31 +-859980069,71537350549419768,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.165,1964-07-06,true,701921630.415,12:24:30.55 +-276968322,47523811902322536,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.165,1991-07-06,false,409389108.127,21:52:31.27 +2125267086,23566147015637484,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.165,1976-04-25,true,1345657905.06,8:37:49.57 +-312870071,31083459210219440,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.165,1959-01-03,false,1423083641.81,18:33:8.14 +-365051256,36140817571454976,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.165,2012-08-10,true,866113962.374,21:49:8.2 +1766031829,69167305200981304,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.165,1996-09-03,true,219371843.862,13:4:5.10 +1145699388,25474222474488424,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.165,1973-10-21,false,986763625.181,11:32:15.21 +1926877015,71690277859639864,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.166,1976-03-17,true,881972426.998,22:37:57.5 +-1273662016,83518918747107824,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.166,1996-01-26,false,365581066.743,14:27:32.45 +-2041431142,63395924531255376,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.166,1971-03-16,true,590451211.585,10:29:6.29 +1762144910,15398465275749058,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.166,2011-04-09,false,1231777076.69,10:12:14.15 +-2043355011,64235551779264680,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.166,1960-08-01,true,1032338895.12,18:55:50.32 +642569438,30512422053576356,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.167,1968-01-14,true,297490623.461,11:54:12.31 +-1519597309,42011814763421400,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.167,1991-07-01,true,1260338158.26,9:5:6.3 +-2007834001,64918303370103872,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.167,1955-02-22,true,32209239.1145,8:13:9.37 +1351176116,4761971343396260,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.167,1962-05-24,true,705664763.807,20:17:36.33 +-1270674083,62891828756147520,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.167,1996-03-05,false,266186322.842,17:32:31.13 +-1108140641,48176808312893880,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.167,1982-04-16,false,527797834.915,20:3:12.22 +50463078,67003313132074568,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.167,1976-04-15,true,939977456.908,10:54:20.50 +1700681958,40308051334908312,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.167,1979-09-09,false,701940608.123,15:35:51.45 +373339621,46199639865489656,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.168,1976-02-10,false,604617512.162,4:51:48.57 +-1594867148,64240632701431736,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.168,1992-02-12,false,733139253.233,5:8:58.3 +-469016008,53657224088122768,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.168,2013-06-07,true,162821513.94,10:16:6.27 +-2132741760,1100427129224786,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.168,1992-08-22,false,1386585502.05,7:7:13.39 +-1867880105,51476614120506080,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.168,1967-08-13,true,1078469027.31,17:20:2.4 +1804104538,54210709052388128,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.168,1964-11-05,false,665377021.28,13:11:32.56 +1189708710,65755562439281952,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.168,1972-03-06,true,264400098.797,14:16:19.34 +1067162947,34244208474621780,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.168,1967-11-13,false,78406610.9568,8:33:56.58 +-31837704,45435146805091752,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.169,1969-05-21,true,304673943.625,12:50:35.55 +173668906,56062309036770184,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.169,1989-02-03,false,1056579370.4,1:57:18.11 +1252126161,79620698211344160,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.169,1979-03-22,true,377253783.018,8:29:15.13 +-1681395446,64883046778948720,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.169,1996-07-20,false,1041440354.86,14:58:2.56 +1704199377,36012670620635688,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.169,1996-08-20,true,847862862.541,8:57:26.39 +-1484869692,34271803691332416,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.169,1954-11-02,false,543876108.758,10:6:10.1 +1238742152,71300661375726720,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.169,1957-05-26,true,657634318.663,19:24:33.4 +355981550,68160351181717608,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.169,1981-10-16,true,1344501362.42,6:39:58.2 +726557249,73587569800814320,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.170,2006-08-21,false,40947042.06,16:20:47.20 +-328207769,79295339560386240,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.170,1995-07-12,false,561464699.616,19:53:45.9 +1609760330,4977618200733512,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.170,2005-04-20,true,1298714510.92,8:29:28.9 +-504681117,64416468856569144,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.170,1972-01-13,true,1035332743.41,12:8:10.39 +740155789,72545029510642816,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.170,1958-09-11,false,754233298.873,4:45:51.50 +2045164515,49558234575422088,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.170,1979-08-08,false,576427759.116,13:3:54.47 +738233445,24753352801295012,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.170,1956-11-22,true,700419752.616,19:50:34.36 +-1080971655,15256020745598280,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.170,1959-08-23,true,135098463.071,10:28:35.34 +-412117253,82294562425404896,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.171,1952-02-09,true,553401113.854,1:13:56.10 +-888932587,4733797262412431,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.171,1988-03-04,false,281380025.233,11:38:43.51 +685604235,34385638236721264,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.171,1975-11-01,false,290570119.165,17:31:38.27 +873043334,49785039885866024,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.171,1984-10-25,false,649596664.41,22:19:24.6 +-1962859684,71713017064640728,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.171,1962-02-16,false,1022181766.94,4:23:7.11 +-969029370,89920605587579024,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.171,1999-05-26,true,1378998221.11,10:1:4.35 +489277488,43558889851490088,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.171,1959-07-15,false,1292710848.6,12:4:23.19 +365610729,66329223052215576,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.171,1966-03-02,false,208530675.109,13:2:30.52 +-1470991765,45105804915627632,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.172,1986-02-13,false,398845521.357,19:6:14.13 +872879802,21875343212983592,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.172,1973-04-07,true,654744289.675,17:28:48.16 +-1172307303,66153252437549600,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.172,1957-01-20,true,1297523104.83,13:4:37.7 +-622580641,68751567860699208,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.172,2006-09-16,false,166747248.321,11:44:47.54 +1951052867,288363287707125,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.172,1980-01-04,true,835981055.166,9:9:47.32 +-2035108375,77187691879684080,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.172,1998-05-20,false,805249786.84,10:14:22.47 +87586680,46678881678848168,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.172,2004-08-12,true,949494612.905,7:21:4.5 +818112928,61051473363913640,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.173,1987-01-05,true,1241123490.57,5:2:20.48 +-83009138,13767040333161340,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.173,1965-09-08,true,1314700061.17,12:1:54.58 +-1379911146,51718634848494392,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.173,1971-04-06,false,746724249.378,8:27:6.24 +1668723660,38502640242994352,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.173,1998-10-05,false,666735594.674,9:49:40.7 +1506960130,33210458437022332,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.173,2012-04-27,true,1250397499.79,17:43:44.24 +2118853071,68293851578159096,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.173,1982-09-10,true,246042543.943,6:21:24.45 +-1863292896,51723277632484160,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.173,1956-04-27,true,761888700.52,1:17:17.18 +143304432,21493506511805756,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.174,1970-04-22,true,867444726.799,7:27:36.23 +96886647,18684237797304496,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.174,1962-02-07,true,147792143.62,9:3:32.10 +-624356726,80112833470715056,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.174,1990-05-01,false,394447334.636,16:7:40.29 +541395185,80550569440604000,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.174,1974-06-20,true,303040749.676,4:33:36.23 +2043633557,9649012914613372,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.174,2009-11-10,true,62597392.8721,21:6:45.13 +-755940822,19557825168464292,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.174,2005-02-06,true,395463271.792,14:44:44.44 +1440400404,9248536396466402,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.174,2010-01-06,true,1426522433.06,5:41:51.22 +-52812591,89933213560481136,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.174,1968-03-02,false,112854279.008,1:22:14.15 +1704941602,49005457934044600,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.175,2012-05-25,false,118112082.034,9:10:45.57 +47857284,4485464481837128,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.175,1981-07-22,true,236982396.042,19:49:38.5 +-2020656059,87155022708862976,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.175,1984-02-10,true,583549077.628,17:47:38.38 +-708525217,58132076068758960,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.175,1990-04-14,false,1067153317.97,18:7:44.16 +-969446348,21416863887969712,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.175,1962-02-08,true,61559841.8067,17:17:51.50 +-2121812244,87214696559052912,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.175,2009-07-06,true,698479969.04,2:55:30.24 +-1518205677,51130676633018584,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.175,1971-01-12,true,680878835.669,1:5:15.27 +1715314209,6300962804974039,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.176,1967-03-21,true,993514075.136,18:42:55.12 +-1456003192,37082754429509784,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.176,1992-07-06,false,1375540164.36,21:54:14.54 +81000032,50850974530172400,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.176,1990-04-13,true,454594242.519,20:7:23.6 +1475936076,80195755672840064,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.176,1961-05-16,true,1230575702.08,11:20:37.27 +1994559009,14887275757851648,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.176,1953-06-14,false,1146980599.66,10:51:49.43 +1358037971,18548958283793304,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.176,1987-04-18,false,157720995.839,10:50:51.43 +-2123061030,37301548671719856,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.176,1968-06-26,true,1217977580.5,20:45:4.49 +1243062799,9876094724937258,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.177,1953-09-08,true,954484784.497,2:51:32.38 +650306047,41080981720191312,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.177,2009-02-26,false,664558062.22,8:10:47.56 +1403627216,74154932570314128,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.177,1972-05-20,false,1034161319.89,7:39:22.29 +1363754304,65914311273005392,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.177,1963-08-02,true,1246249916.54,15:13:40.27 +-1140645226,21795964474514904,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.177,1986-05-11,true,1023866526.55,8:7:8.20 +1316158939,88367609713802992,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.177,1982-03-07,false,1248742395.24,3:4:36.42 +1988091660,73408493191279600,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.177,1962-01-10,true,1359984677.76,9:25:36.33 +159377075,66054731956987128,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.178,1983-07-03,false,711294778.463,11:7:3.10 +137016010,54244607207163376,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.178,1994-11-18,false,968874730.396,4:36:55.1 +508942733,80976682574192832,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.178,1986-01-19,true,329739056.308,15:11:17.5 +1314248051,68101524891459472,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.178,1984-02-12,false,497468216.065,2:17:33.26 +-1453138696,65226137293794968,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.178,1991-04-04,false,954761683.112,22:10:13.27 +-1121078999,53063689861402296,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.178,1982-01-19,true,1213440867.83,14:32:41.33 +1575228811,56630731049904672,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.178,1986-01-14,false,868042701.05,19:7:30.44 +355133682,30085622739483128,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.178,1995-01-15,true,117689393.846,20:6:58.42 +2054811964,65834987997487000,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.179,1985-05-01,false,944310856.554,7:14:30.3 +123363818,17640362116326964,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.179,1993-06-21,false,436930809.275,8:27:11.27 +554632323,21377082823014800,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.179,1965-05-17,false,1008456339.15,6:24:19.26 +-956212777,26017229673793804,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.179,1975-08-14,false,897193962.511,16:34:12.23 +428833919,88645761792718976,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.179,1973-11-18,false,648741983.95,21:41:4.27 +-2124296417,31178173483644172,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.179,1953-04-16,false,869588537.984,15:4:33.56 +-164267039,20414066993697784,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.179,1968-03-16,true,58407345.1606,11:46:23.25 +-306286429,5942624221387551,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.180,1990-07-18,false,732561899.562,14:40:31.8 +-1258518500,85459708963101264,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.180,1970-02-17,false,1252430347.99,3:9:31.35 +-151245710,2284143214982246,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.180,1959-03-12,true,950651707.431,13:46:8.56 +2034712857,22320381437036164,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.180,1971-08-10,false,1075501173.87,17:46:42.31 +-1298452973,70318625524528160,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.180,2000-10-13,true,1086922179.1,19:2:13.25 +-1277262164,63889006183380672,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.180,1979-05-16,false,913947589.552,17:31:51.21 +1879291785,1100728856961945,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.180,2001-01-22,true,114007792.85,22:19:50.49 +1239475687,31056853659981304,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.181,2013-09-18,true,1381668312.04,10:18:47.49 +930312610,15794766616002498,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.181,1989-10-16,false,1128837442.93,3:38:54.43 +276938101,10880793957355294,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.181,1992-07-18,true,1376569413.57,10:19:11.48 +884824905,68957054859556608,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.181,1996-11-19,false,1396028464.16,20:2:54.31 +-672628749,31818082797538264,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.181,1959-11-04,true,681229034.549,19:41:41.44 +-1840879360,16487169235658722,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.181,1992-07-02,true,935666827.521,17:20:36.12 +-2082811902,76867788507889952,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.181,1972-07-21,false,26902802.5552,7:17:4.47 +-1759192923,48148057595242152,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.182,1983-02-26,true,767851854.258,22:36:32.31 +1932563787,27181721079093792,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.182,1992-05-23,true,1245265925.26,4:11:4.23 +-142708498,91688476410230784,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.182,1991-04-27,false,59639717.4864,9:33:35.14 +1837395474,65536721525886824,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.182,1974-05-21,true,415472615.807,16:28:40.46 +2133499945,81224248811409456,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.182,2004-03-03,false,1419208480.9,11:54:14.42 +-632056810,72396725617390032,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.182,1971-10-16,true,335995069.72,13:4:56.23 +-387146815,39672753705676736,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.182,1963-01-27,false,623268473.216,17:19:28.35 +2044354074,38274211609200744,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.183,2001-03-05,true,327164271.632,2:55:7.21 +-1889626821,56352104337906976,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.183,1974-08-04,false,1414097936.58,6:15:38.1 +719081544,20423653518550100,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.183,1980-06-19,false,978856161.513,15:35:3.49 +-609903581,59358619583590384,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.183,1976-08-23,false,1291477921.88,15:1:30.18 +-2027801558,74131577572436160,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.183,1995-08-21,false,48478595.5001,8:56:10.2 +1061931400,1059581822256425,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.183,1999-04-02,false,1141098646.21,18:20:38.50 +842007863,89696948191934800,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.183,1998-08-24,true,341915340.849,5:52:26.31 +337644690,90897468125839808,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.184,1979-05-01,true,343294270.58,2:5:17.5 +1524248368,4125675167802204,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.184,2006-02-09,false,502660224.478,22:41:11.13 +-1094165223,67598911853816872,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.184,1981-09-16,false,1284800793.49,6:24:30.10 +-829153117,21822140237139712,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.184,2003-01-17,true,1098242794.82,19:33:30.21 +389569893,75119258334560672,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.184,1990-06-08,false,31683789.497,2:54:23.35 +-1297649524,46192518752929128,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.184,1969-09-16,false,738317610.953,15:35:36.53 +1719599890,71124671037920432,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.184,1974-11-16,true,1328593286.45,16:16:31.14 +-1432311289,35394879338912176,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.185,1988-07-21,true,1028604112.91,18:29:14.14 +-1344897058,58789315668304392,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.185,1958-10-13,true,1164791584.65,2:36:44.54 +1832955445,81609979003643296,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.185,1975-01-13,true,1420283119.44,7:52:3.42 +-1635562696,36533429833581744,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.185,1974-09-12,false,145721655.492,6:14:11.5 +953437258,83282311817648720,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.185,1994-03-04,false,533396614.606,19:38:36.41 +2041091957,41589037983747440,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.185,2003-04-11,true,1050852901.4,13:35:7.25 +409988641,21083209924628428,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.185,1990-11-27,true,1294130338.04,20:13:41.19 +-733665363,77593614136484960,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.185,1971-05-06,false,547426462.834,9:16:28.52 +-1165655837,76958604011281168,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.186,1954-11-24,false,544349179.349,16:2:18.43 +-797213577,57157852445437288,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.186,1975-04-11,false,1064958798.53,20:1:12.33 +-1156864152,41181859609508048,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.186,1987-11-11,false,1070455674.64,17:42:47.34 +-793726045,63753141068630960,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.186,1993-07-11,false,1067302534.87,21:1:1.55 +-1310241895,33763669704183828,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.186,1962-06-23,true,1419636952.68,19:20:12.52 +850133418,44906538764053432,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.186,2013-10-13,true,427670397.316,2:1:47.39 +-1527246524,70293504845157272,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.187,2005-05-06,true,930858388.184,5:12:41.31 +1962718876,59568647035690112,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.187,1997-02-02,false,851753441.669,15:40:6.3 +-525837869,35467052993374056,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.187,1982-09-24,true,682593681.786,18:34:57.52 +-2142215152,85480828183512720,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.187,1977-08-07,true,108420960.269,10:19:51.57 +1457392247,72551750009053856,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.187,1968-05-16,true,187744075.344,20:57:50.55 +-1143032750,79229332716712944,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.187,1996-08-26,false,403241282.296,1:52:3.40 +1210958038,41019594021767224,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.187,1997-11-23,true,962569536.258,6:21:47.25 +-1065376160,56379015523179512,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.188,1973-02-02,true,91088254.2116,14:52:13.46 +-1676983474,70612130089163336,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.188,1976-02-12,true,409875092.076,9:8:3.45 +-369561854,55189086447662552,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.188,2008-07-24,true,1217422887.14,12:35:21.11 +24133792,23866709078534340,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.188,1978-03-12,false,417651572.026,1:17:26.22 +-1366022705,49668221818390528,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.188,1975-10-25,false,719363980.756,18:11:46.6 +-2138197117,1811337082680504,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.188,2005-07-16,false,679147210.231,18:56:32.9 +-2146333318,71668252263875552,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.188,2011-02-27,true,1029548481.86,17:6:26.40 +1369837359,13326119763292458,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.189,1998-05-13,true,435471251.6,3:14:41.22 +1723222780,57972333405477080,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.189,1963-04-04,false,251662007.439,1:45:52.5 +225162564,43856582040428496,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.189,1951-02-20,false,1267163404.91,11:50:44.35 +-1510428290,43685956934814208,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.189,1981-07-19,true,637054440.17,18:37:32.27 +-514402513,54663717073350528,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.189,1985-03-23,true,1047682825.69,15:28:4.50 +418005188,28882034154482912,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.189,1995-01-07,true,1112067263.75,9:40:16.31 +-62064122,61004095861054832,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.189,1998-04-10,true,1111788384.76,18:57:31.33 +1552530730,33562023551680380,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.189,1957-03-18,true,102559651.39,22:10:48.13 +648191257,83027944942224096,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.190,1964-01-14,false,1167608595.89,7:30:28.34 +1935098651,14064591447734560,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.190,1985-10-22,true,315822692.558,11:27:16.18 +1577441697,70937699298654200,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.190,1959-09-19,true,661511151.274,6:56:4.7 +-76356995,41799002187132816,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.190,2006-04-11,false,779444754.251,19:19:39.37 +-1376475950,8071949437711442,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.190,1989-07-04,true,465145893.834,5:56:29.29 +87904652,16081493084195348,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.190,1990-05-23,false,331647825.661,22:3:16.24 +-836787813,15226742143157402,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.190,1954-03-09,false,182993600.189,20:37:21.56 +1275091759,3678343578339737,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.191,1966-11-10,false,782821614.57,13:30:11.38 +900379044,74161673421035056,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.191,2012-04-04,true,1101161146.77,22:32:24.47 +1417838669,89644262165199472,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.191,1975-08-12,true,671903240.49,15:39:34.18 +-2134914806,59773071953192080,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.191,1978-05-24,false,195403488.568,17:37:20.36 +1099412602,10488497099584164,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.191,2011-06-10,true,933795960.377,15:8:41.45 +-261866795,22777509943376516,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.191,1996-09-12,false,1229815208.11,21:57:54.22 +728580015,42267825928576496,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.192,1976-10-09,true,595705890.512,16:33:27.14 +-483332319,43433844722125464,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.192,1979-06-23,true,1190832544.24,15:39:16.44 +-686805017,20135866284328816,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.192,1971-01-10,false,452272674.243,5:49:9.33 +-997092714,89983452155742176,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.192,1985-02-14,false,1116057663.72,17:14:4.49 +259746365,80338360571248624,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.192,1962-02-24,true,929933161.206,6:46:51.51 +-686386150,26992230717375108,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.192,1993-01-11,true,76558414.7598,11:12:4.18 +1907271889,43740986764819368,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.192,2005-05-23,false,764607177.645,7:7:43.7 +250483,8213839374439762,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.193,2013-07-25,true,1400642535.98,10:9:49.28 +-869998540,19772172398309120,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.193,1955-07-19,false,504088569.414,22:22:57.31 +-1585680971,3631689973163049,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.193,1951-02-04,false,899070714.078,14:23:39.29 +129964386,70358746410087888,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.193,1981-07-26,true,497412695.621,4:50:49.35 +1474827726,60306556024195688,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.193,1962-05-04,true,713218502.96,1:24:7.39 +1178649943,32625174949599836,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.193,2002-08-12,false,5731804.74231,3:48:46.13 +300153218,67806061486229504,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.193,1983-06-15,false,639273820.284,18:18:52.35 +853909498,25261446277234576,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.194,2005-08-21,false,1039040971.31,12:28:44.12 +-909254134,47041358748962728,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.194,1974-03-02,false,429380843.578,15:2:23.47 +-452699482,28770713467168268,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.194,1965-02-05,false,354363478.818,8:26:28.8 +-1863689051,72821753642146720,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.194,1975-11-18,true,1233613811.78,2:35:16.37 +-1548487224,53793835316109448,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.194,1963-04-07,false,675099726.152,16:35:52.51 +-456854482,50740437717501752,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.194,1997-10-13,false,1350532295.15,13:2:36.55 +-1782195930,73050534837009104,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.194,1960-06-17,true,133427793.798,7:56:38.7 +774486485,12948915608711158,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.194,1977-07-22,true,642382061.568,6:39:34.7 +1267907353,3870542100050258,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.195,1982-09-09,false,835045718.786,1:13:57.24 +-1214297920,70149601246330216,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.195,1982-11-27,true,1186594003.23,18:23:34.4 +-1220723161,16845910659736096,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.195,1955-02-27,true,949017609.561,6:3:3.47 +1396215585,42151677527803632,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.195,2002-08-27,true,918214698.123,12:1:24.26 +258084099,70404670392802304,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.195,1975-01-06,true,629370979.323,22:44:1.40 +-1843788640,56079115164954816,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.195,1956-08-04,false,1277983843.69,4:40:49.20 +-636787615,18374973718499084,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.195,1968-02-10,true,1114583933.23,14:51:6.31 +-458744272,85492547894328288,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.196,1996-05-15,false,877373996.025,1:55:46.3 +-1396128375,62727645131552448,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.196,2012-05-15,false,1243487674.49,17:43:44.50 +383449410,78625313700744048,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.196,1965-06-17,true,1287175081.31,9:16:11.40 +1872099265,9721426136582278,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.196,1985-07-04,false,595983698.405,7:5:35.13 +251401569,6335453307650755,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.196,2003-06-19,true,885529460.766,13:21:3.45 +-1563523207,11345127351601112,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.196,1989-04-03,true,852385994.152,7:47:48.55 +-948516980,61200485264993744,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.196,2008-01-19,false,1073692952.98,2:13:52.35 +959480688,41725333769757832,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.197,1991-09-19,false,760915147.271,17:2:3.54 +1941233186,62622164809988888,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.197,2009-03-19,false,1057966350.39,9:42:32.12 +1391180617,2164608554896824,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.197,1985-06-14,false,334182363.874,13:56:50.4 +667615861,32304215758514124,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.197,1981-05-09,true,1406876797.93,5:47:15.38 +-1986583425,52012966844257096,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.197,2012-01-27,false,166460359.698,12:57:20.42 +-2029639897,85649192217153616,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.197,1987-03-25,true,1388148437.93,12:55:1.44 +308642778,89042242557573792,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.197,1968-02-27,true,683095243.331,13:38:25.36 +-1850054609,9615951378861794,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.198,1979-06-09,false,35842248.3727,3:57:44.5 +1898402027,21434387808162356,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.198,1994-01-21,false,303647368.646,4:46:55.4 +-2103511268,36723695064235072,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.198,2001-01-12,true,1074636231.51,22:33:4.42 +-572055614,77283134060665616,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.198,1992-10-01,false,854249696.606,11:35:10.29 +1737875475,14746085401851342,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.198,1960-03-11,false,1065508341.37,13:19:11.19 +2088510426,54002144092760560,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.198,1989-11-02,false,788543732.511,11:17:20.10 +1471924756,59729662212071616,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.198,2011-01-22,false,381175754.835,6:41:58.13 +-2128508401,1074263266062530,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.199,1962-10-10,true,658609535.07,14:48:23.13 +1210271572,26973684131998740,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.199,1988-11-17,false,844829604.546,3:30:58.42 +-991297334,30818092348493096,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.199,1956-11-18,true,633768956.582,4:19:11.5 +2113362737,77815051552047376,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.199,1986-06-09,false,1075475722.32,4:51:9.15 +-359479650,76421424565758544,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.199,2004-01-02,false,317088839.59,17:28:28.21 +-211170782,22900642724168716,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.199,1952-11-05,true,832837847.627,13:13:6.55 +-1143206147,63044114556024576,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.199,1969-06-05,false,1017966171.98,18:12:20.48 +-914942794,67601136893401568,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.200,1968-08-23,true,821618236.834,13:49:19.38 +1332979871,11130905086238936,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.200,1973-10-18,false,137491289.641,7:57:36.47 +-1954116732,34703436328658516,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.200,1958-07-22,false,588595635.795,8:52:43.42 +-2107948274,64881679644223336,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.200,1990-09-27,true,483424178.405,11:1:20.23 +-1417159834,157634868318115,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.200,1999-08-17,false,174387623.184,17:3:20.46 +1052220933,36361592747467000,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.200,1950-05-19,false,197974569.132,20:49:9.12 +495919565,34340026071492648,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.201,1965-07-20,true,1128647003.13,12:27:30.58 +-787647569,25402332110371452,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.201,2000-09-03,true,832564531.54,22:52:44.20 +707967618,29544939239924992,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.201,1960-10-18,false,557489554.585,4:52:43.46 +1407154612,87507420246812144,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.201,1965-03-24,true,1378026227.43,5:41:14.18 +-803144189,78602360040939664,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.201,2005-10-19,false,285528121.644,20:8:39.36 +-1134169639,33557464629928888,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.201,1963-01-14,true,492072670.122,4:18:41.16 +-948145020,85921224500258944,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.201,2000-06-25,true,42079069.5835,9:47:52.54 +1814603894,39231571444184176,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.202,2002-07-23,false,66421526.1443,3:19:45.15 +1029091877,11442631347177708,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.202,1961-08-09,true,823116333.294,2:2:10.53 +-1584533973,6042948641379994,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.202,1952-11-09,false,1108333977.62,17:6:16.20 +1990991560,54237864883637472,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.202,1990-05-15,true,1349910399.47,20:51:13.12 +451511598,43341529355937064,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.202,1967-06-26,true,449973908.069,16:18:25.15 +-784167924,35106838206509252,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.202,1984-11-01,true,38824700.0939,18:45:50.8 +1296579365,40701567542190488,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.202,1973-06-24,false,1143793986.56,17:49:18.54 +-873143968,10443788644448942,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.202,1953-02-08,true,1201807288.72,4:1:25.9 +854826088,90812427902988128,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.203,1970-03-05,false,1136467907.25,6:57:11.42 +430861785,51340111944664952,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.203,1994-04-24,false,445877823.008,15:46:58.4 +-1368860987,79915522869038320,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.203,1971-09-05,false,981256928.856,4:47:16.52 +1511964935,22524895610631660,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.203,2002-07-04,false,1203121776.96,2:5:51.26 +335672934,65732171993223600,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.203,2011-02-20,false,1417092825.69,21:11:55.9 +897280009,60822640414222512,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.203,1970-02-18,false,94476169.3667,1:22:52.5 +1203849834,43082290866925152,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.203,1975-03-08,true,1073929498.97,6:18:46.22 +1494048985,53421478503412072,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.204,2006-01-13,true,368321869.376,15:57:50.31 +1242893510,82269013119192736,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.204,1999-11-01,false,608200662.36,20:30:11.13 +237015020,31545240743681272,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.204,2012-05-01,true,504466868.92,17:28:39.21 +201981707,7069095391171748,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.204,1950-06-23,false,231753841.926,6:44:40.37 +-534641348,61337354389042104,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.204,1994-05-08,false,290030197.137,21:28:8.51 +-1433773041,19562834740092468,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.204,1951-05-11,true,1030452272.21,20:2:55.8 +116452390,35108622320258592,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.204,1952-08-02,false,13010422.498,15:15:54.23 +-811362311,33070320986621552,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.205,1970-04-13,true,1034350127.74,22:23:45.29 +-1660479175,44817950411148696,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.205,1990-02-21,true,170721083.39,3:19:30.52 +705580585,35212631672394068,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.205,2010-10-22,true,1344662805.43,16:26:43.9 +1829894010,64468740841031128,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.205,1997-10-13,false,407850601.624,9:16:14.23 +330361074,19648000010066720,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.205,1963-11-27,false,599243427.282,4:48:52.26 +1888128739,37472073987608712,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.205,1950-08-04,false,1182662455.16,10:34:51.43 +-1818273137,25990300976044628,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.205,1964-11-24,false,668833236.763,4:30:17.58 +1223868180,52103285331625800,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.205,1969-11-13,false,1237967481.76,15:19:18.9 +-988907058,46214359362941520,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.206,1988-08-22,true,1143049012.83,17:27:9.53 +-817124087,84750963968908256,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.206,1996-01-06,false,1281205133.92,7:6:17.32 +1739667279,85920269652671792,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.206,1966-07-10,true,424200138.169,10:35:33.6 +-260984761,31692847441025260,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.206,1977-01-03,true,1111296273.25,16:27:26.26 +1911133355,52596512374121392,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.206,1989-04-24,true,1065749017.45,15:9:39.56 +186965238,16341359576753674,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.206,1956-05-19,false,392032276.473,20:45:25.49 +-1706787078,36926082919269616,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.207,1985-11-22,true,1398370426.92,6:54:6.10 +979211279,66675154211578960,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.207,1958-04-25,true,911064401.223,16:22:31.48 +148451544,89268351809185584,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.207,2007-09-09,false,1288626520.51,1:3:27.28 +-1427689428,6900942259147797,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.207,1995-10-14,true,304111656.248,12:18:34.43 +-1504129669,13603292514625218,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.207,1957-08-16,false,38021035.0878,7:47:5.15 +1898705771,3676911056893542,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.207,1977-08-08,false,424901330.219,13:36:28.21 +-175154200,46233071273675376,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.207,1954-01-26,true,1376696132.52,2:1:5.10 +1623565415,63141648520236224,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.208,1967-01-15,true,716367423.202,22:28:22.53 +597476199,76367532132296480,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.208,2006-09-26,false,1120947475.23,22:43:8.57 +312088977,83457161615741264,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.208,1950-03-25,true,699655120.084,2:30:13.15 +-1828050705,34889366259132968,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.208,1982-01-17,false,415253981.893,14:25:50.3 +978761380,10671759050046146,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.208,1997-04-01,false,1192184811.48,1:6:17.33 +1703689120,89684441527829152,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.208,2000-03-05,true,128982213.326,9:23:20.35 +497918293,63661592407364936,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.208,1967-04-11,false,1011470341.51,2:14:53.54 +197678977,66896998860464456,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.209,2007-01-07,true,637410184.394,18:47:34.31 +-28812161,10618595971671164,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.209,1968-03-26,true,145734619.601,17:56:19.27 +-1832557697,48397788111438552,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.209,1984-10-12,true,1427891339.27,20:54:40.22 +-1068901149,1261700646507806,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.209,1957-05-17,true,247095210.409,22:41:27.28 +-549095386,10592866472487640,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.209,1980-03-08,true,1204080203.71,21:39:44.31 +1478777968,66033558187115528,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.209,1956-10-15,false,333134996.451,1:8:14.21 +304958564,46797422569996784,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.209,1986-05-21,false,444115592.043,2:50:31.55 +599349796,81065835534236992,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.210,1960-01-22,true,793751964.512,16:56:13.5 +1326779979,50340952635418120,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.210,2005-06-14,true,1187602505.36,18:31:1.48 +-1124276551,39891900330154384,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.210,1964-02-25,false,830576451.769,6:53:19.33 +-1554123893,60414412523927544,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.210,1981-01-11,false,881367316.083,15:58:16.14 +-1232147625,18857892908254712,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.210,1950-05-20,false,379806903.459,17:7:2.37 +1163211063,52658162255598456,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.210,2006-01-09,true,294655732.447,13:9:13.40 +-1005651977,49635271790944120,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.210,1979-01-12,false,255152234.42,17:31:22.12 +1099667530,78209994291588208,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.210,1996-01-27,false,777094903.71,5:26:28.12 +-604642262,78560786813429232,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.211,2001-10-04,false,494959319.111,9:34:17.37 +-1536235507,84041385484565376,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.211,1952-11-22,false,445877028.014,11:48:27.49 +49597981,85806180462128960,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.211,2008-07-08,true,289440345.973,7:18:40.34 +-1763219045,2050696563716475,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.211,2011-06-22,true,837386334.91,1:35:25.58 +-2122957557,86172026230605072,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.211,1981-07-13,true,442445013.77,8:14:43.57 +-50680691,75214977209159,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.211,1990-02-03,false,111597185.872,10:53:14.38 +-1930110327,3136401607835842,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.211,1969-09-22,true,376227717.357,8:54:31.40 +1688343976,64388650033760480,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.212,2012-06-05,false,838342898.692,9:53:3.29 +886532721,18276840179554172,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.212,1992-01-16,true,591555304.945,14:27:26.16 +-102412392,80031926162092688,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.212,1959-06-24,true,1054094969.75,5:15:47.20 +576489807,29933778103079088,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.212,2010-06-25,true,1128038770.21,17:31:38.25 +711951017,30285455395697192,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.212,1964-07-08,false,452930421.337,9:36:55.15 +992149978,57558550702801856,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.212,1994-01-27,true,1042820294.43,18:20:38.8 +-1054852412,54851433477795592,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.212,1951-07-21,true,1343866188.54,3:4:26.38 +-889660427,58200985594300816,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.213,1996-09-08,true,170383278.601,5:54:53.5 +281766811,78094070808048864,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.213,1999-09-02,false,1258960841.18,21:31:37.46 +1604836897,7978332091228590,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.213,1997-06-07,false,991453622.373,6:24:47.30 +-2014087090,36915728048413624,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.213,1980-06-14,true,445988584.214,9:38:1.55 +154566823,73824449999505744,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.213,1966-07-24,false,463893152.186,10:41:24.30 +1263002732,30740527001438332,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.213,1970-05-27,false,461164500.829,11:4:29.20 +1374689997,48235663996546528,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.213,1974-11-03,false,416883934.577,1:7:40.45 +440116018,48906136854559656,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.214,1955-09-20,true,516793499.244,12:50:27.19 +1760039387,35425147557566200,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.214,1994-09-27,false,1021959480.22,13:30:18.32 +1056873817,79917109179051456,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.214,1967-07-07,true,512906667.859,5:49:48.11 +-761664267,86311848903001920,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.214,1990-08-18,false,1074707374.82,10:50:50.32 +-129716379,56799215724578552,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.214,2001-06-08,false,724752292.463,21:56:9.29 +1011320417,47455111594218904,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.214,1958-04-22,false,1155277899.96,22:9:15.20 +2139315316,90664439971396256,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.214,1969-06-05,false,683253021.807,18:48:16.51 +-784959959,42919220055128128,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.215,1982-11-17,false,347371572.347,6:58:11.47 +714161594,3714964008170721,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.215,2006-09-20,true,1380736557.41,11:30:56.2 +1769139405,66907807912925080,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.215,1957-04-20,false,90561848.3234,22:4:33.1 +1484588817,90589317582285264,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.215,2008-05-06,true,1400051965.26,8:52:56.5 +704490133,69216601179777976,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.215,1971-10-07,false,90361370.8645,10:15:23.28 +-1754916107,14659930390362194,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.215,1964-02-04,false,1388595737.96,15:44:41.54 +-1872390827,56739144389743728,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.215,1958-03-14,true,157811290.66,17:2:18.14 +826944459,60864253672395576,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.215,2010-06-17,false,858121608.613,13:36:55.50 +-1463658539,38386496460697296,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.216,2001-10-19,false,147903324.581,13:25:20.33 +725343925,6180411802890578,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.216,1994-05-25,false,780590103.182,8:27:30.28 +155683875,50145132971866704,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.216,1983-01-15,false,1033570189.73,16:33:45.43 +2042490502,9660112550799238,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.216,1964-03-19,false,772012154.149,19:53:56.53 +-1059679945,24569626281497016,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.216,1972-06-15,false,1111571010.23,14:7:28.12 +-1990614941,80187971091442928,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.216,2000-08-22,false,644185331.807,15:57:52.27 +-642697203,40978724307250928,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.216,1975-10-23,true,1237036814.74,10:34:12.30 +-1765919447,73667920737829264,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.217,1984-02-26,false,447165975.571,15:3:46.12 +-58594152,27477149551873876,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.217,1997-10-07,false,774257719.266,22:55:15.3 +-286482762,15518167823505346,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.217,1996-09-17,false,958196519.165,11:15:29.46 +-1450351299,79273170068297600,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.217,1983-05-02,true,558496813.044,19:16:33.27 +-170242558,9183123703300312,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.217,1987-05-06,false,113058658.359,2:13:47.55 +1819788538,72401253881260416,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.217,1978-01-08,false,1036596511.9,18:36:13.25 +1714425040,73997113293539984,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.218,2000-04-17,false,437244117.262,6:49:9.30 +-636750581,84869706390207520,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.218,2006-11-07,false,219963737.099,15:3:1.21 +2006449019,59752413140857960,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.218,1961-03-26,false,181679536.172,21:11:10.58 +-1636058078,14147277859171738,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.218,1989-01-07,false,1040829400.62,2:20:6.19 +-1966014205,36167376561694960,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.218,1992-01-23,false,525755486.363,2:18:55.43 +-549604258,63352377844471336,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.218,1993-01-15,true,1100695873.65,6:17:15.15 +1506477724,57620671474397496,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.218,1984-11-03,true,1011625636.88,20:14:26.22 +385331429,25137242407938152,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.219,1951-01-21,true,595480376.031,18:36:28.17 +-233321955,83916545596239456,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.219,1977-11-27,false,99162419.3994,1:12:38.35 +-227914902,16212042874043106,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.219,1976-05-26,true,596813752.325,6:49:57.39 +1464110704,4635593861994353,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.219,1991-02-03,false,401519189.466,17:18:41.28 +-280826419,24165543275166916,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.219,1968-11-10,true,488262461.441,6:17:3.27 +628486402,30301960882668176,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.219,1965-09-13,false,565001305.925,1:11:16.3 +1145928776,10319653376197100,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.219,2000-01-20,false,318925511.579,14:3:55.29 +1867706103,29193885771424296,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.220,1997-08-22,false,459548658.985,9:47:48.57 +-1332783647,13583652891603580,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.220,2001-04-14,true,1041427108.44,8:34:26.49 +1822591950,5148480136648376,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.220,2007-11-16,true,953798252.383,16:26:33.26 +-1330592543,9578232962619464,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.220,1995-03-15,false,1140676890.19,17:51:54.8 +1789453754,20876087710232852,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.220,1963-02-23,true,442795767.11,13:48:43.51 +-20829921,74606512012571936,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.220,2010-03-17,true,1273248876.3,17:52:53.45 +-253129235,91789892487395264,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.220,1952-06-05,true,546749616.045,3:3:26.50 +379734251,15498174157333780,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.221,1970-11-26,false,1017558295.4,6:53:23.21 +285082675,53639941817667880,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.221,1968-01-15,false,805721742.214,5:26:25.3 +801352075,19795503031172864,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.221,1996-07-19,false,585841128.238,21:19:31.23 +-2039244569,83942887650170544,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.221,1993-10-16,true,502102464.231,7:50:24.50 +-504400838,13783340406407558,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.221,1987-07-18,false,1371254552.81,12:14:53.27 +1149207326,5926451748269814,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.221,2001-01-19,true,383390183.915,14:34:41.28 +705326524,15084914864192624,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.221,1987-07-09,false,1369589566.01,4:24:7.27 +-1294958112,51111196563437080,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.222,1955-02-23,false,376303661.686,19:50:46.32 +-1750330053,34154823582129040,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.222,1975-07-19,false,1178140169.94,18:38:23.53 +-771901496,3246773977413161,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.222,1981-03-17,true,1243748280.66,3:12:22.30 +1910910941,71568340180546952,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.222,2003-06-17,false,719131475.333,13:24:18.55 +1456917845,63965150249650208,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.222,1956-05-03,false,1129943041.31,7:34:40.32 +-803445270,37721082783927344,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.222,2007-09-04,false,348100646.678,13:54:55.35 +143403105,49010075680605816,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.222,1972-10-03,false,539723360.832,1:48:30.6 +573392139,4988023261047532,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.223,1987-06-16,false,415030046.416,10:26:50.43 +379189820,34757787656286312,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.223,1996-11-20,true,1177313714.15,22:18:36.31 +-1313512526,36434370869304176,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.223,1998-08-27,true,764468417.477,9:46:18.57 +1297327051,27729027085851228,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.223,1973-03-18,true,113256075.124,2:53:45.36 +774552086,39299629005951272,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.223,2007-09-10,false,234909200.046,4:16:45.12 +-762708867,7356518146462096,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.223,2000-05-08,false,99277137.6467,6:32:40.42 +-188852759,24688412530402212,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.223,1969-09-23,true,454414197.491,10:54:50.27 +1524353073,4825637183298878,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.224,1980-01-27,true,258850803.288,18:42:47.46 +1015423236,79331932481163904,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.224,1951-03-17,false,1303605325.46,9:50:46.4 +-458647814,83984535281541104,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.224,1975-07-23,false,398123498.062,20:45:20.24 +457639852,12962736524418908,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.224,1974-11-02,true,943230961.357,10:40:37.28 +718863048,15086414215492546,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.224,2007-08-02,false,806266181.29,22:30:33.34 +1465448623,87104314933482208,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.224,1965-10-06,true,1278694999.08,12:17:24.25 +408108297,46292135062884608,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.224,1971-03-13,true,564378519.666,21:38:12.16 +1518365010,56573757061443848,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.225,2002-01-05,false,629647799.845,15:4:21.42 +-768038588,32761233590423000,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.225,1973-02-08,false,541092727.596,18:12:54.23 +-1940899097,36286137015629008,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.225,1981-08-26,true,576431138.915,13:1:48.6 +87360663,49448271476716584,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.225,1997-10-15,false,836015161.938,6:12:24.33 +-1508148981,20631016094281768,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.225,1964-06-08,true,27176200.2588,5:36:19.14 +-993627642,75424399572642576,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.225,1952-07-05,true,1413172051.18,14:22:15.17 +118084599,36200539502716960,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.225,1994-01-15,false,862325030.547,4:40:55.21 +847822940,79867015080366400,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.225,1973-05-14,true,1004046432.68,4:23:23.25 +133415125,21631623503319184,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.226,2003-01-18,true,521297893.24,17:40:31.42 +660909462,11057773995792262,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.226,1958-03-21,true,1269598090.06,21:23:9.18 +-1067029858,90719839581490624,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.226,1974-09-22,true,320673499.317,16:53:48.19 +-710364154,7329652057366989,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.226,1968-09-22,true,1334568002.75,21:30:16.40 +-218561559,66168696772059200,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.226,2001-06-19,false,160668792.288,20:55:1.49 +-905848238,54858802073488424,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.226,1966-10-03,false,697666218.558,22:40:45.18 +-2073395477,57271878585781200,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.226,1979-02-19,true,184710742.794,6:2:34.29 +-2107899652,85282316901973008,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.227,1975-04-17,false,1188313640.42,6:38:43.18 +-1831889836,10211164403996856,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.227,1986-07-08,true,974971816.509,17:8:7.42 +-578284237,57055459675361,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.227,1957-11-19,false,177062764.969,6:46:29.38 +1157578748,23421199807819420,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.227,1979-02-01,true,734797152.652,9:31:19.14 +-545692341,9623929515268486,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.227,1964-06-18,true,1311696377.9,5:48:2.33 +2085499722,70397238906146248,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.227,1967-06-12,false,1419651582.16,8:1:6.55 +1348472675,79312158058799952,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.228,1990-03-21,false,1320212437.17,10:54:8.35 +1145430355,12719542478184674,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.228,2009-05-10,false,1366093280.28,21:40:24.45 +-1026562171,91056616410164864,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.228,2001-07-25,true,744722710.61,19:54:26.37 +-1168897150,8549254827402005,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.228,2002-06-04,false,223799517.128,2:9:38.48 +450331820,82760791912887504,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.228,1990-05-07,true,805703461.339,7:39:13.56 +1342271292,63978882230314792,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.228,1978-09-26,false,35558178.9924,14:8:36.47 +-11299753,19919039603911464,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.228,1965-02-26,false,1175486138.49,14:15:58.3 +-886167681,57025951878753632,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.228,1998-03-25,false,810588954.325,8:54:14.54 +-124857531,88533134836090816,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.229,1989-03-19,true,990599142.695,18:16:57.39 +1214712836,16521604728219782,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.229,1975-03-09,false,703116421.275,5:39:34.22 +632157034,72012993302185288,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.229,2013-06-07,false,223084373.994,15:8:54.38 +1565818802,72066419459856256,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.229,1965-02-27,false,32801252.0802,1:47:4.40 +771810600,89631568784136640,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.229,1983-04-16,true,396567159.587,3:44:20.18 +-965736323,22082332034928608,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.229,1976-09-11,true,415917548.896,7:7:21.56 +-1488179560,46092742813967784,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.229,1989-01-18,true,165763287.618,4:38:49.21 +1043256789,55694021178688880,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.230,2003-07-14,false,948533696.366,8:36:29.31 +-1524618793,86810937699723344,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.230,1980-05-01,true,1070297770.42,21:38:1.39 +1419866495,63091683819476792,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.230,1975-01-27,true,1036323950.38,14:24:37.9 +-1594476290,33261225971561236,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.230,1976-10-27,false,1264388255.71,17:40:36.42 +802831564,43715839709227584,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.230,1958-09-14,false,446864996.689,17:58:6.20 +444949450,64178577887663624,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.230,2008-11-26,true,501067822.403,15:52:53.44 +689018510,21437364462466940,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.230,1978-04-23,false,808585392.93,16:37:41.6 +-648837581,57895505682154136,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.231,2012-07-16,false,1375996097.89,9:54:30.19 +-1156306188,17082617713737288,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.231,1960-03-02,false,105844933.752,16:57:48.31 +1638645005,35889030122213480,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.231,1951-11-17,false,596996093.937,17:49:55.12 +1407226060,26737898283529276,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.231,1990-07-21,true,1336925301.09,16:44:44.48 +1082617407,51803153951485192,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.231,1993-09-22,true,1279039256.57,13:50:28.4 +-1442819380,86350202926254928,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.231,1952-10-04,true,427290827.862,22:36:12.45 +442268464,13061286668170804,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.232,2010-09-07,false,689036052.055,3:1:39.1 +1006627091,26798195071056956,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.232,1977-01-15,false,1320987187.12,14:9:57.11 +1800151805,5055284083582823,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.232,1956-08-20,true,524079483.909,4:3:9.43 +-391832783,32060691840960480,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.232,1964-03-03,true,1093606430.66,17:42:49.58 +529996869,77899980774440448,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.233,1988-10-22,false,1359067733.74,10:28:26.10 +-591245736,20676244191357808,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.233,2007-08-06,false,204470777.572,8:30:47.24 +-2134228860,48391808539981600,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.233,2009-02-24,true,548706976.317,13:32:50.15 +-727728216,34951445564808316,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.233,2007-07-17,false,804963834.432,17:53:38.27 +-661147479,1199460119261173,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.233,1961-02-27,true,421594862.184,2:41:54.3 +-1914138930,11889051394291016,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.233,1996-05-17,false,1431841425.6,9:29:33.11 +-1152213665,26031194102560532,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.233,1981-09-13,true,197951388.074,6:36:21.4 +-1545734551,54516452980317688,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.233,1957-11-16,true,1413245574.37,11:37:47.43 +-110459910,50203317033202064,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.234,1974-06-02,false,1334969566.0,1:47:38.53 +605763756,32569697864566068,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.234,1987-06-17,false,741048345.909,17:30:14.27 +2081692703,18827245175476972,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.234,1971-07-13,false,860348837.293,8:11:29.36 +-2113468984,81422785023221872,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.234,1986-01-23,true,1007538678.88,22:17:25.39 +322372463,23854087666934048,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.234,1999-10-25,true,274485715.879,10:57:26.28 +2129268771,30362699932956480,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.234,1978-02-19,false,1205004086.4,13:17:40.49 +-2113538541,71490943874536656,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.234,1989-08-25,false,371735763.641,14:16:47.40 +1784450435,30601527645665812,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.234,1967-04-15,true,1118855520.38,16:28:13.11 +-2004607097,32084139741886484,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.235,1986-02-19,true,266845564.807,13:12:52.12 +-627687211,33597750395112900,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.235,2005-11-13,false,284004476.793,13:13:40.45 +2009723951,31032128333616180,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.235,1961-03-04,true,884568540.686,12:12:11.21 +1006804221,63540501456600464,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.235,2002-07-25,false,236730353.706,14:7:29.11 +369113410,88995363022145184,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.235,2008-02-13,false,1055383387.19,18:47:14.29 +-145441827,1773271917094000,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.235,1956-10-02,true,363556172.826,17:4:38.10 +1350935072,47692937282939256,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.235,1995-01-24,true,1346016764.61,8:56:38.40 +548570769,48717150540608512,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.235,1976-04-16,true,91024608.9492,8:12:41.23 +1658478126,89882894063521136,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.236,1966-08-01,true,161972980.655,3:39:29.55 +308595260,44031209752449240,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.236,1991-09-18,false,105134909.699,17:17:1.29 +1236468855,87179294913046496,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.236,1963-08-08,true,810937851.512,10:39:29.34 +-1072988341,13771633264639426,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.236,1971-05-04,false,178960558.367,5:17:13.38 +-1704065411,47509698478799656,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.236,1993-02-23,true,1339842492.12,12:30:14.28 +-1061965564,47578897917722912,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.236,2006-10-09,true,1034047805.35,8:10:11.38 +1657164741,34088296956740680,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.236,1974-06-15,false,862579337.63,19:51:12.48 +-65126285,194584819966044,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.236,2011-09-14,false,573877162.801,1:48:31.18 +-1719334477,13343240046787952,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.237,2011-07-13,true,218599549.42,5:13:25.33 +356190374,53600200202604352,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.237,1971-03-27,false,1166338227.77,8:53:47.3 +1317447991,81708132938151664,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.237,1958-08-03,false,1032192000.06,8:41:8.39 +-209643457,75819783731819152,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.237,1995-04-17,true,729668535.602,4:43:3.50 +-1615671907,80156906720777024,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.237,1953-01-11,true,693913491.908,5:41:5.14 +-1675003408,21902745442256632,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.237,1984-01-15,false,1343975270.54,11:4:43.11 +-1013874874,6302460870630083,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.237,1981-05-27,false,997663461.831,16:2:12.37 +486107509,2115285035930849,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.238,1959-07-20,true,377707408.678,1:58:20.56 +958086408,74840399126929680,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.238,1952-04-01,false,1221102305.9,5:45:25.40 +596644258,51134995765393264,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.238,1996-08-09,false,1425596263.07,10:19:9.2 +-419352746,17612652551611434,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.238,1996-06-22,false,181504154.362,9:11:24.54 +2017288613,29662934302832960,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.238,1978-04-07,false,214991031.242,6:58:9.36 +-647303104,29917093610890272,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.238,1997-08-16,false,48134999.5318,18:28:5.16 +-1836899233,46961340031402968,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.238,1993-08-08,false,1248343733.4,6:33:6.1 +-224565664,35538594271677472,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.239,1973-02-23,false,1311048023.7,9:15:43.46 +-170871249,76726778461821456,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.239,1954-05-24,true,1031575457.4,13:21:28.1 +739553852,88285828871540848,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.239,2002-10-18,true,1001524867.17,15:3:20.23 +-1528190073,32342086012678624,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.239,1958-11-25,false,539867944.514,17:48:3.38 +-1465080184,55011285589038776,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.239,1950-09-10,false,1377158957.27,14:41:10.52 +-664288810,25879189460487160,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.239,1987-06-24,true,370492422.873,14:39:33.55 +1137340064,3741882412351549,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.239,1970-05-25,false,853658275.405,20:41:24.25 +-1186545811,61157811682682960,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.239,1994-02-03,false,360449435.264,8:6:36.47 +-106736347,70795977563488408,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.240,1966-02-07,false,1942103.02782,15:55:24.49 +-841080057,52959926450618320,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.240,1960-11-27,true,1299785835.87,21:58:7.13 +514831635,44058185797310888,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.240,1992-01-14,false,1048768545.13,17:19:16.18 +-503620988,81273253128720896,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.240,1969-06-07,false,70596315.0538,15:24:2.19 +-191695839,85626557001567152,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.240,1968-07-24,true,377100289.065,15:56:20.2 +2079586307,32400673715955180,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.240,1991-02-17,false,911057792.937,11:20:20.19 +1951474327,34342124096759616,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.240,1993-10-04,true,23963963.9923,21:42:43.16 +1798226541,4813541085750313,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.241,1964-02-21,true,1314634383.89,21:43:19.56 +798879342,47152420489381160,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.241,2001-05-24,false,823507224.633,11:22:13.27 +-535885902,79941840880184192,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.241,2001-04-17,false,374802811.35,1:24:4.45 +-962357784,70836728454929616,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.241,1969-09-11,false,303799035.727,6:39:27.43 +1563748770,78491822809506256,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.241,2013-02-05,true,754022118.514,9:46:6.6 +-462038964,63010978945454416,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.241,2010-06-05,false,88946181.7883,3:6:57.53 +927103880,26067608621408852,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.241,1966-04-02,false,1183142605.82,14:30:23.24 +1460062185,90338564382025600,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.242,2012-06-12,true,246652660.099,12:11:22.50 +-881652740,57002646975241176,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.242,2008-11-01,true,826425792.964,13:40:26.19 +1914114041,63441077549052272,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.242,1951-08-15,true,253204703.472,8:55:17.46 +1165584788,57976214813029928,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.242,1989-09-12,true,987240503.957,12:27:43.46 +-2047015177,30731497562630072,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.242,1985-04-16,false,1342278.23054,19:8:29.13 +983728495,84924018002199776,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.242,2005-10-09,false,406521427.181,16:34:36.1 +465747224,81090393218319344,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.242,2003-05-20,false,1248690834.93,16:52:40.45 +-233352592,16529064106919404,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.242,1957-10-08,true,98259613.6656,13:40:41.15 +1618933356,76486073304414000,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.243,1982-11-04,false,1097205345.93,16:13:26.38 +-806665039,80545077085038224,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.243,2003-11-15,false,1011804064.78,15:12:19.48 +1335145084,44233952511038424,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.243,1977-05-20,true,178669671.802,13:35:13.42 +-1536206278,86207284036436144,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.243,1967-11-24,true,1363809844.78,10:45:44.46 +490335408,26552192877312124,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.243,1988-07-20,false,142514914.332,13:7:9.34 +-263860492,532333651482808,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.243,1976-07-19,false,220876558.208,15:15:55.8 +-2028742186,80711249660106000,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.243,1960-07-08,false,1386148961.02,18:42:14.17 +-1950055467,60366150059196216,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.244,1954-05-24,true,298924086.952,20:50:56.11 +-1097945002,5630004101278567,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.244,2002-03-16,false,1346990119.68,11:51:49.16 +-1866795181,7196704949498583,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.244,2006-07-19,false,1310438248.58,6:39:13.12 +-1946448683,85190053280738432,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.244,1970-02-19,false,167695432.994,10:43:20.42 +231335650,15998716980323462,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.244,1997-05-22,true,707341498.964,2:21:15.12 +-161571017,75546347808403120,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.244,1970-06-21,true,514406543.603,9:17:47.44 +1089511445,1786650150915410,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.244,1966-02-11,true,844191285.61,2:27:19.22 +1827999631,11429753800858338,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.245,1957-01-09,true,169992806.318,19:18:27.26 +821306291,10927004748243590,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.245,1978-03-10,false,972267084.21,2:58:21.42 +1594735350,43597374719281368,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.245,1987-03-13,true,172588105.675,11:27:52.37 +1685691598,31504070483982532,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.245,2005-11-07,false,1234805812.04,22:38:26.38 +1755540341,60275340277323768,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.245,2011-04-16,true,1385629672.88,10:37:40.13 +-2121252402,36642464291826856,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.245,1978-02-11,true,968085712.598,10:31:33.13 +-1131739744,7335246013140583,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.245,1955-08-10,true,192343181.267,16:18:45.28 +-1607761171,56433545767180384,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.246,1977-09-04,false,314235309.546,20:48:24.56 +-1369657484,75174417520402560,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.246,2009-08-07,false,778411057.883,1:11:35.50 +2104134242,9394915851478150,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.246,1984-04-03,false,1091314262.4,5:53:41.19 +-1605213228,79224644400811920,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.246,1979-02-08,false,1063322815.19,4:22:22.44 +-1580984534,84981449454395040,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.246,2013-04-12,false,383377211.993,18:5:57.6 +-1024078078,4888387336860140,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.246,2007-11-24,true,1224255066.34,8:35:22.50 +301322239,7105841327099024,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.246,1990-02-24,true,1399869198.64,5:24:51.9 +-313023830,32120161651054468,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.246,1957-01-04,false,998521823.299,4:45:48.45 +1892238548,27603867401414136,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.247,1950-04-08,true,871339851.725,16:30:57.18 +1375717100,36644795825303336,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.247,1957-11-16,false,1180582525.06,2:4:30.17 +-249885333,27899699069429588,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.247,1991-04-26,true,787795005.795,1:29:41.35 +-1999728099,80943955241095104,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.247,1956-11-16,true,247525491.181,17:56:42.16 +-1255233817,43608996364370672,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.247,2001-04-19,false,1413272705.36,17:16:58.30 +1899667166,36090240310057328,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.247,1988-02-04,false,621056290.43,7:24:28.57 +-541405292,5441640423464284,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.248,1958-04-22,false,406720928.141,10:50:15.29 +1067241823,46076328367623856,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.248,1999-06-06,false,63730317.5997,2:27:17.54 +927889943,51738250553604088,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.248,1989-05-11,false,1344274903.44,21:3:3.41 +946659653,88008336774830576,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.248,1957-08-23,true,1205450232.4,16:13:38.29 +262790885,60971026186038976,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.248,1955-04-03,true,605538366.062,15:21:41.8 +271818319,75292906552126144,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.248,2013-04-15,false,985914571.884,9:22:43.44 +406383580,51394170599421320,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.248,1975-06-19,false,658597163.811,21:3:9.20 +-976193363,65098784938961176,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.249,2012-09-12,true,724858064.262,11:47:25.9 +775116266,76886891472062576,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.249,1978-05-27,true,1061974102.46,4:22:52.46 +925958889,71211877528614184,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.249,1995-11-16,true,991597464.69,6:27:52.41 +-1235117062,25284868715723468,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.249,1987-07-13,true,1181437620.01,12:55:23.47 +433018070,86663588758623568,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.249,1994-08-02,false,688966751.248,3:57:19.57 +-1945311792,45717838431314528,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.249,1978-03-25,false,1371993065.74,5:14:8.3 +1034720775,45469113833356936,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.249,2003-11-17,false,687572056.669,2:2:17.24 +-944601898,30827119907647264,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.250,1953-07-01,false,612709848.852,4:51:14.23 +1819037556,47954160572005400,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.250,1981-10-05,true,766442793.434,20:16:11.51 +-1651842790,42043376722374976,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.250,1987-01-01,true,1390433596.93,13:24:7.54 +-248371614,33276116432853544,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.250,1979-03-27,true,776577786.255,15:13:25.58 +-1270272167,23103100508230112,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.250,1980-08-21,false,1340886751.74,22:54:46.34 +-339536977,62135916581686408,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.250,1985-09-27,true,1110164659.62,10:57:38.52 +-1905106390,10151099147151842,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.250,1983-01-07,false,770168623.84,8:45:16.2 +-647904849,12329639970037350,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.251,1993-10-11,false,161537156.765,14:18:10.33 +843160892,36547370426041496,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.251,1961-04-02,true,862656682.514,8:4:27.56 +-138256444,20359709670071572,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.251,1953-08-14,false,1033631959.56,22:38:58.30 +-1419850301,37901309113788712,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.251,2012-11-23,false,547387015.548,18:11:3.37 +-1325437562,76780561989507760,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.251,2007-04-20,false,1152324765.36,14:29:42.57 +-1481442405,47240922873956272,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.251,1952-02-11,false,1395914560.43,9:20:14.52 +1423254725,88836675370390672,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.251,1976-04-05,false,672294601.764,3:48:25.53 +1851980056,27347236261129556,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.251,1987-11-26,true,1074628011.69,9:11:18.1 +-1441914331,64636430680012296,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.252,1951-06-18,true,1243306144.88,4:8:48.45 +-1602844387,32313275713186264,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.252,2003-09-03,true,1214092910.98,22:47:16.52 +-549908223,37606653485297104,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.252,2004-11-22,false,1325715259.48,13:11:21.23 +-1680165560,7125848418747925,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.252,2001-10-27,false,741827816.83,11:46:30.25 +-2065741290,67546122582840240,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.252,1996-02-23,false,1386728175.45,1:1:38.36 +-424374849,51289420776196136,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.252,1955-01-02,true,929754265.931,6:15:15.45 +-2039705524,72196871103944512,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.252,1967-02-26,false,1041594813.35,2:13:19.54 +77737605,72495654991027392,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.253,1978-09-23,false,1103175551.64,19:7:36.45 +1936895268,61341247695238288,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.253,1953-08-12,false,671594822.478,9:22:24.29 +-2050697924,70409220831486752,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.253,1995-11-18,false,349772754.633,19:12:18.16 +1962752934,50475535350711808,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.253,2007-07-22,false,1255493978.37,9:28:39.35 +-1206567737,28043728072486696,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.253,1988-02-02,false,959493460.791,12:34:14.31 +-1421747570,82416864930753952,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.253,1967-02-05,false,921048703.533,8:5:3.57 +1299097404,4858218951003105,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.253,1993-11-05,true,364919056.942,22:51:58.5 +-1002524155,70826808031453240,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.253,1991-01-20,false,1137895813.47,11:7:24.26 +1251061144,31158142001254648,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.254,1998-01-14,true,1371400578.84,10:40:42.47 +-387926382,64459183135287328,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.254,1987-10-10,true,1183061503.68,11:46:50.13 +-718979450,36839476308305784,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.254,2010-09-18,false,652059289.79,17:57:53.24 +-1464585128,18059360472898672,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.254,1980-03-05,true,1398310214.83,11:35:57.48 +917364736,4921406612998738,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.254,1968-03-02,true,1356497074.62,16:24:13.3 +475718709,28203033054776352,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.254,1970-06-09,true,123598236.227,10:54:56.8 +67590972,56231587670879400,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.254,1985-07-21,false,1300094687.2,8:2:18.19 +-1927061210,83121931250363936,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.254,1975-08-04,true,1206314360.07,20:7:13.45 +759152183,59643766242585952,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.255,1965-04-01,false,1389082570.57,16:15:58.52 +740523145,1661794054917007,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.255,1967-03-18,false,712830248.176,1:6:12.52 +552583529,70147253345173656,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.255,2008-10-13,true,184289028.348,20:52:20.58 +1943188677,24633184937129472,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.255,1997-05-20,true,75642866.3043,2:18:43.39 +898703800,53864806619421496,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.255,1960-02-18,false,913863718.307,14:17:49.19 +389825490,24258054498703044,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.255,1990-11-12,true,1188332702.99,18:1:40.58 +-591404408,79760798877343008,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.255,1997-01-10,true,910090070.6,14:51:31.55 +-1109640459,5555208693944627,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.255,2008-10-08,true,213112117.243,4:16:28.25 +-2113478853,88172008888123360,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.256,1952-06-12,true,1384207509.73,22:9:32.8 +-550389860,45152860341107664,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.256,1952-09-12,true,129418638.551,18:54:55.1 +1034383092,77183928484427872,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.256,1986-11-24,true,849353289.185,20:45:7.5 +1817698865,61267059102228776,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.256,1962-11-15,false,1190889116.53,4:36:52.29 +627227272,31260766093871628,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.256,1997-10-14,true,724240747.249,8:14:4.21 +-321874812,21538310615535676,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.256,1962-01-02,true,971837076.598,21:27:33.39 +-1828879700,56093331579723184,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.256,2001-08-10,false,51898869.5874,16:16:38.46 +1877688775,46391998405239264,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.257,2008-05-19,false,86232052.5541,14:50:15.46 +-1839783576,62990500706019944,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.257,1972-10-10,true,1079451709.31,19:53:44.25 +-848539567,59294254136545888,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.257,2003-10-27,false,1372377671.27,12:33:14.47 +-1011136882,47825083987688128,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.257,1977-08-26,true,445745836.866,20:49:31.32 +117766830,69628028274238664,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.257,1954-06-25,false,1269944516.61,7:28:42.24 +-865843259,3436421570380380,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.257,1990-04-15,true,1324420020.97,2:6:49.35 +231828276,68652673261432352,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.257,1970-02-26,false,457531764.196,12:48:4.6 +-732415782,44633194719363696,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.258,1993-03-15,false,997231559.512,8:14:50.13 +-2140687612,17482315324771052,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.258,1992-08-22,false,46530341.4965,22:32:57.11 +-1617159079,28890134173627076,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.258,1998-09-09,true,1272361538.08,11:56:43.7 +-1559771663,32135328942664868,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.258,1955-01-12,false,779794172.867,10:50:7.27 +-72251214,47549113503359960,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.258,1956-09-09,true,1245464863.46,4:16:55.56 +-867951035,54613359078094080,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.258,2008-10-24,true,584005643.433,11:11:45.47 +946704285,42146929419401392,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.258,2004-11-23,false,195581224.373,19:58:16.57 +1723527971,29547936584677428,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.259,1972-05-16,false,1328405608.35,7:3:40.31 +-335659790,24461226350976176,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.259,2012-07-09,true,916094425.014,7:47:30.56 +-1731595602,50188838120842808,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.259,1961-10-07,true,192224862.469,20:18:27.40 +-73654784,68762841014882584,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.259,1975-06-22,true,402707526.822,6:33:50.26 +-18580322,205047899438796,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.259,1986-03-07,true,1256983426.9,9:53:20.27 +1259113401,37477285047963672,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.259,1974-07-17,false,84188961.5566,14:3:58.55 +2065604156,23883618205806920,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.259,1975-02-01,true,1182713689.3,13:31:35.39 +-1512245875,62395494990737344,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.259,1982-10-27,true,263070260.33,10:34:10.31 +-698764013,75305286540758656,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.260,1956-11-27,false,984571390.643,16:55:34.14 +1054307808,8276123976675922,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.260,1974-09-24,true,952118286.326,22:38:47.8 +234815222,75211677654600448,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.260,2008-09-12,true,619531130.129,15:49:6.38 +558558389,46593775125361912,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.260,2004-08-24,false,653385898.785,16:36:15.27 +1017747956,34782971625838500,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.260,1965-05-16,false,1285498906.37,22:57:16.53 +-1181818074,78847792538296464,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.260,1981-04-07,false,877431211.021,22:16:2.40 +1739240898,91134015251767888,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.260,1966-09-14,true,576059576.562,12:39:24.6 +-1813020637,85680667481257296,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.261,2011-07-11,true,915168077.263,15:51:19.34 +35656832,13350110394561034,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.261,1988-05-10,true,1012954169.14,3:42:43.35 +2010722380,79631232344520640,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.261,2009-01-13,false,1004400727.12,18:29:51.4 +1270048225,66962424994041144,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.261,1981-06-04,true,1349754513.44,8:32:6.15 +-1846010457,30940950249276264,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.261,1967-03-10,true,23545410.2839,7:13:30.34 +-1446849037,58063425424561224,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.261,2003-09-09,false,651180623.471,9:23:34.35 +807660044,47184228228285832,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.261,2012-03-02,true,437168398.945,3:24:58.27 +-373205773,46279817230676992,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.262,2007-11-12,false,482372101.963,19:58:44.49 +-1881759279,887486531502356,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.262,1974-11-11,false,730412425.065,11:44:55.36 +-1006123295,81914374878496528,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.262,1973-06-09,true,436279583.764,5:24:7.50 +-391632818,90288626618247248,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.262,1953-02-11,false,596137743.721,3:39:36.40 +1980117310,56868379808182632,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.262,1973-11-24,true,308284698.513,17:50:8.49 +1060763467,18996754185292544,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.262,1965-03-21,false,1278880906.81,18:43:8.33 +851519458,73272231938071776,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.262,1984-02-22,true,1066240381.77,11:9:44.2 +432565357,9370435497123574,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.263,2003-04-02,false,566100511.76,10:38:52.44 +286792371,57697387053945736,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.263,1963-01-24,true,186810784.797,12:4:55.26 +1047087318,39848621590953552,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.263,1975-10-04,false,1414117492.72,5:10:26.6 +-134043710,67550162233225208,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.263,1985-03-17,true,1103851634.63,12:52:4.19 +720900417,47440041660926984,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.263,2013-06-13,false,1250833616.76,4:34:49.9 +-996471673,11952135046728376,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.263,1961-11-14,false,1347816553.34,14:37:25.32 +152674598,59224724109931712,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.263,1978-10-24,true,925454129.625,15:15:27.33 +-1658119338,37934218717459848,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.264,2007-10-05,true,1365861892.29,6:15:56.33 +-502378171,47536460391274760,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.264,1951-06-04,false,612631815.272,5:57:28.24 +281006833,22118035473408432,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.264,1967-11-20,true,124720276.33,19:54:23.6 +394610522,89830869222481120,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.264,1967-09-27,true,634602126.628,22:17:22.53 +251101422,16999121371923744,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.264,1962-06-01,false,327270651.217,10:49:2.51 +-1247336135,55182826098652728,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.264,1956-11-13,false,1204818882.68,11:55:55.50 +1278666912,83002102705068672,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.265,1974-08-02,true,437216229.178,11:37:6.13 +-1914080785,78634730856812480,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.265,1988-11-02,false,793429138.099,21:53:5.5 +-985185157,77903622748000784,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.265,1977-08-03,false,282712495.209,5:41:8.31 +-500151157,46704221328480176,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.265,1952-03-23,true,384143494.569,21:24:42.14 +-310229104,33039166820712868,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.266,1952-10-13,true,635406377.737,5:45:23.7 +1865330459,75266782621782416,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.266,2008-09-19,false,1129879060.43,8:20:37.49 +802425897,29874769727926336,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.266,1991-04-08,true,899523696.178,19:47:6.44 +-866691510,26727037550362728,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.266,1982-05-25,false,672040678.302,20:5:38.28 +48731993,72900890377158720,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.266,1973-06-24,true,485667512.035,13:50:21.55 +-1352077038,29207597179218168,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.266,2006-11-04,true,969961013.547,21:51:18.56 +-614162298,89352502686835920,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.266,2004-03-17,true,1327813204.07,3:5:13.6 +668636784,83701472726810736,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.266,1951-03-15,true,579209395.549,2:12:29.5 +1140574695,79006662119031760,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.266,1994-07-02,true,753966608.383,16:30:56.12 +950551279,60443527307100776,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.267,1973-01-27,true,157112783.45,11:12:7.26 +2096297656,19201649088474104,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.267,1995-02-24,false,1369750959.26,18:5:46.34 +1458015586,47314474186420184,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.267,1992-04-01,true,164070446.035,21:18:25.3 +-2015764984,46835403755773496,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.267,1993-04-14,false,1266589245.92,4:14:29.45 +50545206,33727068759526300,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.267,1970-05-13,true,687189231.063,13:25:15.2 +2083056634,9871843506056130,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.267,1981-08-19,true,370660799.084,5:31:56.56 +901059438,8456949639544248,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.267,1987-02-02,false,1261184018.48,10:10:41.46 +433781774,87975988679127200,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.268,1958-08-07,false,1184746360.29,16:52:20.19 +-1910272529,77861426490501424,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.268,2011-03-04,false,840767682.285,6:19:31.30 +-153206782,52491888914464224,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.268,1981-10-27,true,463872037.049,16:37:43.40 +430781676,79022234790950816,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.268,1980-04-23,false,428221205.426,3:42:36.10 +-638408595,14495395418258054,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.268,1969-01-22,false,124321278.553,8:52:53.15 +-339279691,74660994511518688,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.268,1973-09-03,false,445455105.078,22:12:49.45 +-1638104196,77836364816896432,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.268,1965-01-01,false,336051645.872,7:28:34.38 +-113838731,29991136256294576,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.269,1991-11-21,true,614157489.556,10:17:51.33 +511994800,78754136529822864,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.269,1961-01-05,true,467558599.568,7:48:19.37 +-364380928,7893151982422897,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.269,1971-03-07,false,1113521951.57,6:25:43.41 +961739267,23484121287948116,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.269,1970-02-15,true,126575884.923,18:38:56.50 +142740737,59687017372049912,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.269,1955-10-08,false,596231143.174,20:13:50.11 +1300853097,59498938260247192,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.269,1977-04-13,true,357960220.031,19:40:45.13 +1506045135,34949122958336852,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.269,1961-11-05,false,466158248.781,3:51:57.28 +2096370550,70741110393129952,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.269,1957-01-11,true,622610212.9,14:17:19.17 +191070955,8788248741336003,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.270,2013-09-25,true,476963851.769,12:41:44.28 +936589114,51379535267476624,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.270,1958-07-15,true,486873246.834,1:57:8.24 +-877953955,33287184187810888,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.270,1983-09-01,true,399872004.785,16:22:23.3 +401315808,41130967370401096,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.270,1957-04-23,true,1107547429.34,1:3:22.26 +89273149,34424738617750632,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.270,1951-06-01,true,898271831.881,21:52:38.19 +144626,82208201432134256,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.270,1958-06-26,true,16528019.74,12:31:12.55 +1644694010,2556531570332139,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.270,1994-05-16,false,1355997594.01,17:31:25.29 +752940152,50276683335721864,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.270,1954-02-25,true,749220127.993,9:49:3.32 +2005282232,18092834733115104,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.271,2009-07-20,true,1427061811.45,4:14:16.32 +-549272743,17745731750302628,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.271,2004-06-25,false,209366853.852,12:34:15.16 +-699789625,35885561519360820,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.271,1998-10-04,false,496170762.452,11:33:3.38 +1571345843,59144594761066408,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.271,1982-08-26,true,1348737572.55,2:41:15.23 +-400422699,37399960437344776,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.271,1963-03-18,true,1135423356.91,20:51:27.8 +1156934273,23958436373271604,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.271,2008-01-03,true,241551813.383,11:19:28.55 +-254448124,17237161679588732,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.271,1954-06-21,false,286100266.851,9:49:11.58 +1411883373,1561237280053872,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.272,1959-09-11,true,289727388.288,21:4:18.8 +1780156136,58255766937629400,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.272,1968-10-10,false,905790930.962,11:2:43.36 +-1812050177,18934758960840580,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.272,2007-06-18,false,591498057.738,21:21:49.37 +-1599516165,88987503591969072,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.272,1988-08-09,false,758533032.757,15:41:31.28 +97298382,64330797514205760,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.272,1998-09-21,false,490325736.628,21:30:33.10 +11556069,37781813991998288,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.272,1957-08-14,false,228713030.929,13:2:2.21 +2123867791,8197199422422549,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.272,1982-10-02,false,539482889.589,2:5:4.29 +-1660032813,45895653434103976,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.273,1967-01-18,false,512007275.851,12:50:37.52 +-667289722,78948134217695872,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.273,1978-03-20,true,772486846.427,9:42:33.4 +-1050239470,74444298501821408,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.273,1952-09-21,true,284377290.672,12:39:57.34 +322750312,20731766460554160,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.273,1975-07-15,false,1044791797.89,8:53:20.29 +1668909754,51970050384354768,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.273,1983-06-25,true,1347984539.56,22:44:52.11 +1668199758,34999794271185336,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.273,1970-06-23,false,1113787877.68,11:30:35.47 +-2030846413,10911424454876386,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.273,1999-08-14,false,1325156164.28,15:8:4.43 +-770568758,34661124776496108,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.274,1973-05-05,false,305941918.765,19:32:52.42 +-29990273,19236034785492144,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.274,1960-03-05,false,104296761.031,3:37:16.33 +489702506,36939130674485856,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.274,1951-04-23,false,507453029.577,19:5:42.37 +2057279019,48330270254874648,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.274,1996-01-18,false,1251367846.67,13:21:27.46 +2042180254,23968832295500392,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.274,1976-07-16,true,843907048.412,18:15:34.14 +-1897469765,60792729284423576,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.274,2000-03-05,true,1038323571.13,6:19:37.8 +-479753126,45476603635990224,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.274,1950-04-15,true,1372110378.41,2:48:14.49 +1472762344,12225076057083034,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.275,1998-02-11,true,1060392999.38,10:32:20.44 +-1284499679,63139237143726272,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.275,2004-05-12,false,1391619309.17,18:55:22.37 +1956552424,83500698847405184,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.275,1959-01-11,true,161530523.249,7:56:24.51 +1241743644,34549694293885000,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.275,2009-03-10,true,760718557.932,22:48:45.44 +1979240029,22845551705856944,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.275,1982-08-13,false,907197222.323,21:54:26.32 +639944002,53554025470721264,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.275,1950-03-10,true,396159956.139,20:25:13.24 +1164539703,51724471174455472,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.275,1952-11-09,false,205729386.171,13:16:17.24 +-1650225916,89618323961409952,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.276,1980-04-25,false,1145155412.46,4:57:12.48 +-272610716,37478855813168704,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.276,1999-11-18,true,659018413.663,18:16:14.54 +-260665550,91792233788052896,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.276,1980-10-25,true,1382968423.69,8:1:39.44 +-1956249008,5718799062070487,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.276,1999-04-10,false,340096657.099,19:14:19.28 +1302484291,8333045486014638,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.276,1990-01-02,false,467774070.371,3:36:39.56 +-2100713934,85012466931288160,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.276,1961-11-02,true,993945403.039,2:4:5.42 +1689001902,61856671330332560,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.276,1969-08-07,true,174039170.691,20:28:2.17 +-1164414942,59776488493433912,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.276,2009-02-09,false,325476927.834,6:2:11.53 +-966509929,50047970502960024,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.277,1989-11-17,false,647563480.61,10:55:51.44 +-1892153158,61272527828414248,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.277,1958-04-07,false,65351002.646,7:16:34.30 +-1526297582,18159489560726140,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.277,2007-11-03,true,1011286864.48,20:35:27.12 +-1828662978,28581411153780092,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.277,1993-07-12,false,950314364.178,5:36:54.43 +-1876393814,53074434417546264,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.277,1965-09-15,true,700254122.057,9:38:15.24 +1179922137,36060604187325880,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.277,1995-10-20,false,445954450.802,8:56:9.37 +-638895282,43423376623928504,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.278,1961-06-23,true,476109942.292,11:1:8.36 +1688151227,14495893819392492,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.278,2012-02-13,true,147956127.335,6:50:52.36 +265210461,90805854271735552,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.278,1999-07-01,false,796234770.153,12:49:23.7 +-132759983,39668261592956056,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.278,1984-06-09,false,1364851284.62,19:43:56.48 +-144704336,53288839832440952,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.278,1967-03-01,false,946793772.299,5:47:6.35 +-893395041,52948417930811512,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.278,1979-01-02,false,699771873.641,6:52:49.5 +781952975,55726299522805048,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.278,1963-08-05,false,363578016.158,21:9:33.38 +-1722656040,62315549384791808,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.279,1976-11-02,true,386718815.015,12:50:16.55 +1384827311,48455993257892160,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.279,1979-09-04,true,1280605127.56,9:52:33.46 +460170551,15386580624182282,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.279,1977-01-15,true,142654969.734,4:12:47.29 +970988561,73957717873720736,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.279,1989-02-06,false,159466767.816,18:52:49.12 +-661719228,60757056980257608,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.279,2001-04-13,false,149609974.154,1:22:8.57 +165048802,15622482695762054,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.279,2008-11-23,false,1424658693.73,21:21:47.49 +1230346660,74457860572744128,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.279,1966-08-17,true,898575629.925,9:42:44.55 +1640545556,575717661669621,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.280,1950-11-19,false,733963533.434,21:46:36.21 +-199605842,1061873376990658,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.280,1989-01-23,false,660681913.728,9:44:33.9 +-782758506,91287649036824000,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.280,1968-05-03,false,314511446.235,8:54:6.41 +-2144850698,64537234835838632,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.280,1989-11-15,false,780337607.607,14:46:5.11 +-1766578037,90675228758987488,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.280,1955-05-20,true,1000136568.1,13:45:6.23 +615393636,71088416273469848,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.280,2004-10-19,true,957166418.708,16:32:52.8 +-568854948,29469019907342776,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.280,1992-03-20,false,1182935083.0,16:57:14.54 +-1743941374,19082142368887776,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.280,1990-06-06,true,386007586.486,13:10:48.28 +-825463629,87563883268584800,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.281,1972-07-01,false,946991763.677,3:55:6.23 +-244852834,18292501859427340,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.281,1959-10-03,false,1340368300.05,16:39:33.13 +-131150970,54072374856074648,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.281,1998-10-13,false,1018195378.82,10:39:8.55 +-1014334701,7501834076383027,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.281,1964-02-11,true,113331010.057,12:47:37.11 +-1494232766,6174799058417459,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.282,1967-09-13,true,538279013.319,13:58:8.55 +217472940,48795482097328640,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.282,1986-08-06,false,668114780.015,4:18:18.28 +1899619812,13030883634913054,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.282,1991-11-23,true,1200780283.81,12:50:50.24 +-1799969384,78989410001249408,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.282,1989-06-25,false,677977145.174,22:55:27.24 +-245601590,58005709802397888,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.282,1980-06-15,true,1182570860.65,14:34:12.45 +-340893076,31696251149221284,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.283,1960-03-03,true,256414962.958,21:25:30.30 +86224271,12888071947518822,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.283,2011-09-22,true,1031639854.86,5:53:23.40 +-1108809951,35048396027539264,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.283,1967-07-07,true,1141079347.11,6:3:15.56 +1818692320,51684301951460416,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.283,2007-04-14,true,1190440265.2,5:51:38.18 +-730450273,88084726439354384,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.283,1985-09-10,true,713223815.267,12:48:30.25 +-1175683221,62846891018682040,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.283,1972-04-24,true,1101345246.02,14:21:24.47 +-156892508,50988788774313248,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.283,1955-06-18,true,15610641.9079,11:15:19.20 +-569999684,39259271431838264,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.283,2002-04-23,true,1280740219.5,10:35:14.57 +258884383,53469787648067440,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.284,1994-04-22,false,1363783438.83,15:35:45.44 +-169000580,4678080213317632,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.284,1983-09-04,true,84203413.8509,2:51:39.31 +-169381190,79965756338464256,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.284,1966-01-19,false,1095863981.59,18:26:35.11 +750395213,81032121560273376,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.284,1969-03-22,true,149281050.458,2:12:25.41 +1742189058,28837265988234068,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.284,1975-03-26,true,1416034335.07,7:15:22.9 +400635207,39792416561681584,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.284,1968-05-14,true,17881690.778,16:24:17.20 +242551598,18822217780204840,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.284,1972-08-24,false,1078247886.79,16:25:16.51 +-1840467852,69788289678765304,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.284,1957-09-14,false,1162090654.61,14:38:41.44 +-1258617242,20165977969782120,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.284,1957-08-08,false,1009957901.76,3:13:32.24 +-1069215008,68117006015844976,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.285,2005-07-20,true,606175395.825,14:21:55.21 +1845926244,70911450316496184,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.285,1995-11-11,true,1367546202.7,16:41:28.52 +-786164945,52311849687802872,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.285,1976-10-10,true,952026443.727,13:17:36.14 +661372215,43284632005830824,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.285,1976-06-01,true,632021457.745,15:18:47.47 +-1743339672,51142682457601760,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.285,1976-10-19,false,121492762.582,20:30:50.17 +1828065027,57099804055441544,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.285,1974-09-03,true,549783807.251,21:40:7.10 +-973904857,76818639483774224,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.285,2013-06-24,false,1361595510.31,10:27:1.14 +-791879525,91588533280905056,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.285,1989-10-18,true,1389297921.27,1:54:38.32 +347499293,11364147238698190,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.286,1977-01-23,false,1268668285.81,5:20:40.3 +-479993506,11169422451787162,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.286,2009-04-19,false,541709469.817,8:23:36.19 +-855931264,9521216520639366,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.286,1970-06-24,false,547369076.203,6:39:39.25 +1543596272,15165522920271340,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.286,2008-07-04,false,949077234.319,6:16:44.17 +-1881109401,35168761029580096,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.286,2007-07-11,false,684694592.618,7:9:23.16 +-393090554,26619685155198156,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.286,2003-07-10,true,207042948.542,15:13:34.45 +1150975623,69567342063070032,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.286,1972-01-23,false,1342204397.98,21:51:45.41 +-1977407894,26400029237309500,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.286,1984-01-26,false,1078149923.56,12:26:57.14 +-1136996950,24963787186597488,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.287,1965-05-13,true,325808706.126,13:58:23.7 +-1472403326,90916317877748384,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.287,1974-11-07,true,1387706572.34,11:32:34.44 +338106671,56733324215429,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.287,2005-11-16,false,625698700.496,19:49:11.30 +-997875239,47558153972725432,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.287,1987-05-24,true,68399566.8175,9:28:53.33 +209321631,88018200921942576,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.287,1957-11-15,false,140652812.409,19:37:10.38 +-2073920601,87984957803246896,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.287,2004-07-21,false,1110874433.75,16:24:17.18 +-1288382518,45208665677185824,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.287,1960-10-14,true,119953181.839,7:10:55.11 +364257191,11339878971309374,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.287,1959-04-19,true,871555252.454,1:32:22.4 +-1700562996,86852502017559808,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.288,1981-01-14,true,1236897160.19,22:19:2.16 +-166729885,56168383835074336,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.288,1970-06-04,true,86375811.0011,17:55:7.22 +-415849775,29970273006206220,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.288,1956-08-21,false,1133007963.03,21:48:55.23 +-293651449,32573269996449248,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.288,1957-01-01,false,1383441005.35,22:5:3.48 +-1987789748,75196887215348000,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.288,1962-10-23,false,144444053.458,12:49:27.41 +-486329954,28736053511251640,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.288,1967-09-11,true,31300934.8714,4:13:43.22 +-179604906,20707385252415140,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.288,1987-09-14,true,229876914.892,20:1:23.24 +793132322,25771220678656492,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.289,1953-10-11,true,118515611.218,12:26:4.26 +152322267,60605389293275616,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.289,2006-08-16,true,645360248.05,18:40:47.33 +1417130076,34778078222596528,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.289,1995-08-24,true,476224003.722,7:38:42.13 +-796683902,1566525039546419,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.289,1987-07-14,true,143501865.731,8:10:27.29 +1450360461,72838065120730496,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.289,1980-01-06,true,196840344.002,20:19:14.3 +339335370,38933275957839480,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.289,1999-11-27,true,61053008.3005,18:40:11.52 +-28193680,33163792471268372,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.289,2004-06-27,false,1260582392.54,9:46:55.9 +-1734842746,41948793238104280,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.289,1953-09-01,true,1178003994.35,6:47:42.21 +-1272480516,23894949508366940,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.290,1982-04-04,false,926725654.103,5:40:36.51 +1956022825,41608341475142000,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.290,1995-01-18,false,1012725268.45,5:56:36.27 +-319511254,2437037281457469,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.290,1970-03-06,true,304508892.191,14:4:38.23 +307055494,10136294645352356,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.290,1978-01-14,true,123873518.766,12:2:46.55 +988201128,25734643203046360,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.290,1950-07-24,false,81158499.5416,7:41:25.40 +-1899102571,51077320067814352,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.290,2006-05-15,false,646323035.262,9:54:53.23 +-216239143,32827273624887092,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.290,2006-07-16,true,1050615925.41,7:10:18.17 +1961042244,35175735930874768,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.291,1971-07-14,true,156961918.721,9:46:14.53 +1582127546,63276463041814384,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.291,1989-09-24,false,1373584829.65,14:44:18.11 +-157028716,32730593217430612,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.291,1963-05-26,false,1286953439.79,9:6:10.57 +-365525198,26017833113367756,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.291,1986-07-11,true,1003613352.72,4:36:47.42 +-2067148101,79793714905933184,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.291,1958-11-05,true,705612548.47,3:32:34.17 +299738744,56227601196815248,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.291,1952-07-07,true,397726120.932,1:23:48.43 +1439595556,66034842661350920,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.291,1990-06-22,false,1044062305.04,4:19:3.47 +-983650476,29907847884112724,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.292,1951-02-08,true,1250768872.57,4:22:6.48 +-78606602,69153713869250616,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.292,1980-07-26,false,398635171.181,16:24:53.5 +-565554454,44039226969108400,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.292,1959-04-25,true,745581902.428,19:53:35.26 +792179569,40238531042900440,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.292,2012-10-09,false,136798306.289,5:36:41.42 +-915684274,76829331567082800,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.292,1986-08-20,true,545048865.109,19:48:40.47 +-733072943,90829235866783136,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.292,1962-10-13,false,612648703.708,21:58:24.21 +-854037023,51867371951832320,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.292,1974-02-15,true,1036133258.57,1:52:31.11 +-2107260799,51071468296993232,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.293,2013-05-02,true,440645004.779,21:6:40.14 +-2042717897,38958720619535072,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.293,2003-04-09,true,1136300974.37,13:13:23.34 +937349257,12580443484902482,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.293,1957-04-22,true,853596651.526,11:1:55.21 +-336962480,4339919147816079,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.293,1972-03-21,true,1115637902.27,11:48:10.20 +2059752257,67047359039698072,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.293,1956-03-13,true,94449491.9342,9:3:48.4 +575338947,25948278165068576,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.293,1975-07-18,true,1422823575.59,19:32:15.35 +-199643904,67748576718111296,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.293,1981-08-06,false,1388085588.3,15:40:44.37 +1510086342,41139240909036064,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.293,1975-03-23,true,508267952.981,2:27:20.33 +-2136539915,49992094369207160,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.294,1995-05-23,false,1229210170.12,3:12:33.57 +1091972025,27091336436588596,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.294,1982-03-22,true,675049688.609,12:11:42.15 +629400155,19936063171254660,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.294,1974-10-27,false,977813877.853,21:51:13.17 +-685741016,65171358167090944,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.294,2013-01-25,false,918606837.565,21:4:38.26 +-1508394092,9384705466373970,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.294,1995-07-16,false,817667017.707,3:34:23.23 +11731013,77953578120386448,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.294,2012-09-04,false,543832189.335,21:32:55.2 +1879655261,16211947319490960,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.295,2009-11-05,false,651872416.157,14:26:8.20 +-367538887,15728652712971868,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.295,1964-08-03,true,970311943.729,5:50:55.21 +-1888634786,24221281643046288,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.295,1991-02-04,false,100609386.151,21:50:32.25 +-365838069,90327703570961024,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.295,1979-05-04,false,1062092938.05,11:48:34.3 +-568825866,67002712767646800,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.295,1956-04-17,false,626432539.244,15:2:25.50 +822115073,24356277720053536,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.295,1994-11-24,true,1241171615.38,12:9:11.44 +-1158451288,9976060262912820,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.295,1978-06-16,false,1207267615.0,1:25:28.35 +-964883295,3856149957968056,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.296,1997-03-03,false,536778632.334,10:40:5.50 +203399803,6243463696688722,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.296,2000-02-17,false,15298929.8355,4:53:27.31 +1537620798,84140440370271696,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.296,2000-06-23,false,829260411.402,21:51:43.31 +1027033557,41888890450209432,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.296,2002-05-04,false,449025895.918,13:20:18.15 +-528164060,4813177851438520,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.296,1989-06-03,false,743970979.233,6:33:14.39 +-851111269,79842995508155184,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.296,1984-06-12,true,757058124.031,13:26:58.17 +-1006171355,40965310562823920,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.296,1960-09-14,true,1270904896.24,12:9:5.29 +845054896,15781681744323768,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.296,1954-10-13,false,1282041665.23,8:3:49.10 +-1147718537,70868936496694528,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.297,1978-03-21,true,452345720.549,8:25:5.35 +1890004978,86512325834215168,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.297,2003-09-23,true,127125441.999,18:18:23.38 +-105956428,2137096864378777,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.297,1964-05-16,true,99705203.6735,8:13:48.35 +1248482524,80827278686147456,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.297,1971-06-15,true,1123554576.11,8:58:49.40 +-343133238,8450913568954839,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.297,1968-04-09,false,1039782049.76,6:18:2.44 +-734242490,84653926055852592,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.297,1975-09-26,true,455609533.756,22:19:52.9 +1900479242,62102473574024848,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.298,2004-09-01,false,827291948.513,2:28:5.3 +-1272269783,50853723509002352,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.298,1952-03-26,true,799489372.109,9:24:46.18 +-571374963,73479522579780400,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.298,1976-05-14,true,1168082905.63,17:34:51.29 +-989030568,46984829152196936,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.298,1957-11-07,false,888782719.945,19:55:43.23 +-34114117,30800162776709684,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.299,1980-06-05,true,187185164.464,20:16:4.36 +-79863992,20142137469226160,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.299,1965-02-05,false,163614291.596,20:28:33.23 +-725111302,30936540555691796,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.299,2010-09-02,false,1267417980.18,13:58:14.14 +1210502539,16026826945379440,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.299,1978-03-13,true,871645118.784,9:45:11.35 +2126735126,33693363444319480,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.299,2001-08-18,true,1035583244.32,13:25:51.2 +1087698391,72234563961232224,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.299,1962-08-26,false,764212599.901,10:58:14.48 +1602204985,37709326710195848,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.299,1974-10-05,false,178843006.594,15:28:7.9 +1500800980,41496971138143472,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.300,2000-09-02,false,872939850.512,9:23:48.8 +2051006482,6510280955567155,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.300,1970-01-26,false,40794770.3219,5:34:1.9 +431283704,79352801351483440,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.300,2006-06-27,false,1266369878.32,12:43:29.10 +588563596,75006038122974368,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.300,2001-10-11,false,458298020.322,11:24:55.58 +-1186674443,69106053864670208,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.300,1951-05-09,false,594511816.173,21:19:26.28 +2106804013,67478183939518640,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.300,2004-01-22,false,1266945483.03,9:48:18.48 +2110612470,5104628666282465,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.300,1958-08-21,false,931130677.376,20:3:6.53 +1710943461,85074366357444512,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.300,1951-09-09,false,137467768.121,7:43:41.30 +-825230377,32301578768264520,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.301,1999-06-15,true,1270254865.05,3:12:58.11 +1178998820,36549324427038208,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.301,1999-09-17,false,1245193548.65,3:25:56.19 +-723738926,47249254300103640,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.301,2009-10-15,true,820381839.88,12:37:49.41 +-1837768481,29245236441218888,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.301,2009-07-17,false,742710257.521,3:1:46.37 +200017050,42800128084439904,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.301,2007-03-17,false,793571925.982,7:4:41.38 +-1315736905,68782347192609088,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.301,1964-05-09,false,527458803.728,4:40:49.44 +-1680350678,89898274783323104,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.301,1976-11-04,true,1324883670.29,19:27:17.27 +-822315238,22163325237896836,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.302,1983-02-10,false,20337529.5855,1:48:51.6 +-489940493,84167659228939392,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.302,1966-07-25,false,753095394.282,12:20:11.1 +1922385667,1069723133037465,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.302,2010-11-21,false,1042643667.56,9:17:5.28 +-1342671390,31867708504794604,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.302,1960-06-18,true,388297471.137,9:23:2.37 +-656152760,39094258704886744,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.302,1970-08-12,false,1046224404.21,5:24:40.13 +-1264813236,59527662760493520,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.302,2011-06-23,true,283935708.409,19:56:4.14 +959242480,90681230805559456,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.302,1989-10-21,true,162734171.854,14:56:34.20 +-1192953491,65965092970821312,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.302,1970-08-22,true,1178723721.46,6:37:51.51 +-1959634118,70765466589676728,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.303,1967-01-26,true,1419380265.75,17:2:9.58 +-392812418,86184132546665712,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.303,1971-07-18,true,94804298.6015,11:9:53.32 +-379073349,82760426424418368,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.303,2012-07-13,true,938962214.929,22:26:51.6 +1892075887,13835944569289410,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.303,1957-11-13,false,797841331.95,15:22:45.27 +1786097450,80748409267792400,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.303,1967-03-06,true,1128617073.84,5:40:37.9 +-780972023,54218746241462912,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.303,1957-08-26,true,1408348272.93,19:14:3.53 +-531754027,71879114253581824,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.303,2013-09-11,true,508804429.159,3:5:38.20 +-1131881014,15489425624398286,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.303,2012-11-11,true,1057421366.61,15:9:8.52 +345855343,6979742438424064,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.304,1981-08-22,true,322482019.249,12:18:6.19 +-186144254,57223405970510272,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.304,1997-08-19,true,90102649.6638,13:58:3.41 +555291308,48214020661111368,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.304,1969-06-10,false,213895151.41,16:21:21.29 +-709469654,30248522369623172,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.304,1951-03-12,false,983707230.944,2:54:22.43 +1785159629,11092578215921972,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.304,1972-02-08,true,870498733.549,4:35:25.48 +-663159457,39816689152909168,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.304,1979-10-01,true,46460148.7825,20:39:51.47 +1944869793,44125080096452600,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.304,1951-05-13,false,147655675.181,12:46:52.19 +1402157731,15605893632601550,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.304,2011-09-20,true,507890190.234,2:11:22.30 +1848143217,61309501625227984,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.305,2012-03-11,true,1144901133.22,13:28:22.18 +546809689,30438588342497916,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.305,1983-05-03,true,1104600540.0,19:53:31.52 +-158062478,15182205651026422,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.305,1983-04-09,false,35085819.4094,8:42:44.36 +279879877,2316095100823808,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.305,2002-06-22,false,245773794.404,9:50:25.23 +-278248452,56463602905162904,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.305,1982-07-23,false,998097406.925,2:21:46.27 +-608418533,89869644255671856,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.305,1988-02-10,false,563604489.944,13:44:25.13 +619363016,91591476898473600,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.305,1989-09-20,true,285219748.085,21:55:53.13 +1945099327,29956291846226820,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.305,2003-05-13,true,246889068.794,20:56:4.58 +-802998835,82605160487395984,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.306,1950-02-07,true,210618570.418,10:12:53.43 +-1013430240,17072156557313998,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.306,1997-06-26,true,175553512.445,5:23:23.24 +1970600400,49767687070890088,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.306,2003-01-03,false,304747802.408,17:27:15.42 +713354873,90010929960231280,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.306,2006-10-27,false,626048240.16,6:40:44.29 +-680887259,28978646354138268,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.306,1968-09-12,true,825679635.571,14:32:41.12 +305111755,40013361588997704,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.306,1953-02-05,false,1171344126.32,19:10:15.7 +-55038829,38504970072737280,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.306,1978-11-23,true,156577176.793,1:1:57.39 +267199507,78670865791986944,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.306,1951-08-19,true,787671425.038,22:6:44.28 +-477878597,75368083596554368,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.306,1958-05-24,false,969771079.079,1:30:58.50 +-973064189,78024081485620496,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.307,1973-06-15,true,537510475.994,17:35:31.8 +132748163,25980139498726256,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.307,1967-07-14,false,1169737521.35,8:55:43.36 +-1442930090,47720251561577736,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.307,2013-01-14,false,436343081.193,10:11:12.47 +-393521730,61517693112699784,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.307,2001-04-13,false,903358909.216,12:40:7.4 +1732719548,56249817744975160,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.307,1956-07-24,true,342450147.267,16:42:43.21 +-1046219653,67995767817292032,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.307,1989-01-27,true,725906896.01,5:17:9.13 +-294023802,87690691264440816,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.307,1968-02-15,false,457871014.959,20:18:5.53 +-1182300291,50071675633913344,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.307,1990-05-02,false,81385083.8466,19:42:18.27 +1364214128,29114854927791820,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.308,1954-09-10,false,777226402.537,19:33:40.48 +581186745,33040742301114052,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.308,1995-08-25,false,910448611.523,13:41:21.16 +-1865231168,25166658850361376,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.308,2004-03-11,true,628633064.092,5:2:41.43 +2103786215,71786756185023888,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.308,1972-04-09,false,31394417.931,4:4:28.16 +1119203926,82858451924910960,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.308,1963-09-22,false,64171156.4925,10:27:44.31 +-789656103,22995862084281876,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.308,1997-08-11,true,842733601.079,16:23:13.16 +116897283,42376190331464960,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.308,1975-11-21,false,666973723.306,10:3:40.10 +-1788798777,19003618961311448,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.309,1954-07-16,false,1048046902.08,11:20:10.36 +-1691138075,26849512321010032,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.309,1959-09-19,false,1033359753.96,2:50:29.26 +-865816695,17772968564947558,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.309,2005-08-03,true,322456437.934,12:20:7.3 +-533029367,61363985568562352,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.358,2008-07-05,false,1179669534.19,11:28:52.1 +1765499344,60739400345174888,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.358,1992-02-10,false,975527943.133,21:40:14.57 +1642004883,85099376415121888,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.358,1987-05-04,false,574726852.721,12:26:56.8 +539980226,89319567949794480,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.358,2003-11-17,false,338701195.03,6:31:8.30 +-1076942895,6344721885451766,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.359,2005-04-20,true,740085428.25,10:53:16.47 +-2086642889,22700346188926720,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.359,1965-06-27,true,1241922353.54,6:1:55.33 +911257949,69708378760498696,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.359,2008-01-27,true,983100954.933,20:34:8.17 +1449928148,10025980607019796,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.360,1964-02-10,false,700382455.146,10:33:3.2 +2039931075,55399789656255928,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.360,1954-09-06,false,1357577837.81,19:37:43.11 +843482084,22431534341035048,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.360,1987-02-13,true,426697672.945,17:5:33.36 +-717060474,2495139994040545,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.360,1967-02-18,false,666866586.342,4:33:12.53 +1881377000,16852470079286426,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.361,1954-08-21,true,838685240.064,3:53:9.35 +-2031916112,5419936243724636,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.361,1981-02-17,true,106001531.442,20:11:1.54 +2042154335,57783775762321952,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.361,1979-02-20,true,555709327.771,6:20:12.8 +-183660831,34494516613503456,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.362,1955-08-04,true,356658439.688,11:19:57.45 +-2061435208,90657264491000256,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.362,1985-02-06,false,979157179.056,20:40:30.36 +1968937117,45967805442715416,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.362,1971-08-16,true,1399321843.92,8:3:22.34 +783920708,62005159328695520,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.362,1954-04-05,false,1306828462.45,1:43:41.25 +-1818997914,80797557757257488,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.362,1950-01-21,true,616800815.874,5:27:12.46 +-1171022092,27239279855089264,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.362,1954-08-07,false,243624445.78,15:29:14.28 +413644255,45713609102350440,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.363,1982-08-07,false,230385035.693,20:22:40.56 +-2113971531,51442771895008280,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.363,1966-02-10,false,814783470.236,4:26:49.33 +-937545597,70335855041888408,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.363,1964-05-14,true,1371382891.28,18:40:4.17 +-580773861,77373071333144672,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.363,1991-02-24,false,607103370.279,17:52:33.40 +1188198756,36724775733460176,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.363,2005-04-25,true,361451695.732,17:24:16.47 +1465545256,87389524019294032,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.363,1959-05-09,false,1341468834.59,3:8:10.2 +2034025049,87579735662979328,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.363,1956-03-16,false,11980404.5222,13:1:10.11 +-1098913141,60168749508529880,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.364,2008-10-26,false,1020941091.8,1:28:56.43 +-1223021290,87716606364263872,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.364,1974-03-21,false,1135381769.14,11:41:38.5 +1098273719,73790484710094080,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.364,1978-05-24,false,765129099.456,10:23:30.10 +-2082207202,31382639987945536,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.364,1979-09-06,false,1034563224.75,13:7:8.7 +-1897277270,67500720326168504,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.364,1990-05-02,true,168856722.525,15:26:51.9 +-1202890191,57255026687802888,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.364,1955-09-14,true,851564747.334,1:28:42.15 +1520434853,78844593879976064,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.365,1979-03-26,false,414284525.178,13:53:11.15 +1815650537,80040989172937056,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.365,1968-11-20,true,1276620778.74,9:47:11.13 +-148094144,27912699063074132,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.365,1978-10-02,false,260735929.413,20:9:18.31 +-1164524743,25923299131665880,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.365,1974-06-05,true,159496216.132,22:8:14.57 +-1641440800,4120045831880417,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.365,1953-03-24,true,1117045193.14,8:31:22.11 +-1113180545,51384894644521328,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.365,1981-04-21,true,244457940.842,8:41:9.16 +904027671,21520514129141976,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.365,1963-01-27,true,78771883.7785,9:14:36.49 +-153563463,64608472833507200,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.366,1971-04-08,true,1325160263.12,1:44:24.19 +-1414081147,57018521530653008,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.366,1976-04-06,true,520295840.231,9:32:8.14 +-883088755,65283552633401608,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.366,1977-01-18,false,540795207.546,14:13:31.20 +-1177416920,20296370895364896,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.366,1960-04-07,true,1034927467.47,8:2:38.44 +976445190,23388559439070548,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.366,2001-02-01,false,854836750.19,2:32:2.22 +-1323631860,84063612170958480,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.366,1979-09-19,false,374873060.906,9:42:14.40 +-838698463,3123676107892029,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.367,2003-04-05,false,1185185937.46,17:53:51.2 +2097894587,11764365147811164,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.367,1954-06-19,false,86229583.4434,18:13:23.38 +-558238463,89097864022753616,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.367,1975-01-16,true,581886337.548,7:39:13.47 +-2080528108,87447734534423856,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.368,1957-02-17,false,1359405088.93,1:9:39.45 +-1664603900,17159716248569528,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.368,1975-05-06,true,30210598.2194,11:23:39.17 +-27575866,68183196597964248,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.368,1994-05-08,false,172983631.935,15:25:57.26 +-1409274272,24119130032728760,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.368,1951-01-26,false,1241281772.19,14:45:22.1 +-518104611,73367524756906480,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.368,1958-07-06,true,518669091.033,11:39:14.54 +76539474,84058372484158688,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.368,1977-07-01,true,1322324259.77,22:12:17.37 +505917666,36400494212696160,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.369,1982-02-20,true,301118240.461,18:37:42.39 +463859978,25688818650357792,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.369,1971-06-20,true,145988201.311,15:7:22.14 +-282279284,41374367428730624,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.369,1978-10-14,false,1147426872.56,22:32:6.18 +1349874707,36938569105049536,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.369,1972-08-04,true,836508747.049,11:35:58.33 +-982502736,60626356371498464,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.369,1987-01-09,false,1367114525.95,8:23:39.49 +-304461202,55234518915628616,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.369,1958-03-01,false,513293769.742,8:53:7.54 +1948558218,64160049651484800,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.369,1998-10-20,true,267649846.117,11:33:35.18 +-2080469073,6072427693895567,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.370,2008-10-19,true,1065662784.2,16:35:20.9 +-618406016,85550824705625456,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.370,1975-03-04,true,424164749.537,16:28:21.18 +869352414,77702438327885664,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.370,1976-03-23,true,568311716.063,2:37:21.58 +-41249091,73951164291399008,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.370,1986-02-13,false,990370929.442,22:5:5.21 +1383322511,22898978298026896,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.370,1964-09-06,true,1408227161.35,18:7:27.7 +63615112,3798845835589192,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.370,1950-02-01,true,787094127.99,10:32:17.32 +532375709,74858050633739168,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.370,1977-05-02,true,429187065.565,15:39:16.33 +-1800876273,92185277477588496,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.371,2012-05-05,true,946002093.793,20:25:13.25 +-927058444,4755472386730086,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.371,1999-02-10,false,1059236763.75,13:41:2.35 +-485165681,22457947857261700,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.371,1978-10-24,false,896003650.333,6:29:28.16 +-1212627857,84700456415207472,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.371,1981-10-11,false,797707749.394,13:29:9.31 +-1486340394,70390911280810488,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.371,1987-03-06,true,452036943.357,19:41:52.29 +985333497,32050935607702928,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.371,1996-08-25,true,33984132.4834,20:43:6.7 +539227940,5454786888707482,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.371,1963-08-23,true,867807247.564,7:49:17.28 +-377090221,41103702174256128,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.372,1978-06-15,true,842430001.018,20:33:38.49 +1265618285,41723873061552056,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.372,1969-01-26,false,1279976557.12,10:36:30.48 +-2125783718,51880336047067832,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.372,2005-07-16,false,419931274.638,3:17:50.21 +1619349833,37261525972941232,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.372,1954-08-07,false,1013453278.67,10:43:34.20 +-909930595,16995552240411034,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.372,1978-10-03,true,117479353.1,21:5:49.12 +-2003228795,83877106939617600,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.372,1953-04-03,true,483344812.94,3:54:2.8 +1250144780,90903481177990928,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.372,1973-10-25,false,779212170.263,15:14:38.24 +767725586,29328905037028700,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.373,1997-03-22,false,759741973.062,5:41:8.15 +-1574910846,39233711854586656,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.373,1985-07-11,false,1284903740.41,8:47:58.47 +1218319447,31823109991979652,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.373,1961-08-01,true,1150386218.27,21:32:11.31 +-728215683,31542505297726648,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.373,1968-11-08,false,1173278129.46,6:45:27.11 +1680471253,84511192387524880,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.373,1963-02-14,true,1355669881.13,4:40:25.51 +-971742640,8486599441133384,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.373,1959-10-18,false,856923093.065,13:15:45.7 +-833597657,56895870091492792,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.373,1992-05-17,true,825971348.165,15:58:6.51 +601069803,19295112755067844,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.373,2001-07-14,true,141566803.304,5:24:27.16 +-126156672,78960777672413248,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.374,2003-07-06,false,1326467535.18,2:30:1.48 +-345763437,80170006830958576,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.374,2000-08-05,false,1071315368.54,8:29:1.45 +1565798996,65379210611502176,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.374,1998-08-01,false,1124088130.17,18:55:23.47 +1609465513,11732719633987430,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.374,1953-08-17,false,505471853.293,10:8:23.18 +-1595560198,38746793675761560,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.374,2012-07-14,true,822806673.263,4:56:6.39 +901417446,41639014774515352,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.374,1950-03-17,false,858974256.71,9:42:2.56 +-1290817690,85421143449806544,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.374,1974-08-03,true,183952177.144,21:21:42.43 +-934295632,76415513812329312,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.375,2012-08-18,true,59010870.8308,13:58:2.34 +-1150341251,21283134432587592,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.375,1950-04-18,false,603158757.028,12:47:57.6 +1811840380,71889581999923152,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.375,1991-11-12,true,488528684.163,22:52:44.20 +-577296678,57248696255255448,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.375,2002-08-06,false,674319959.266,9:44:31.11 +-1022549346,86297981207509440,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.375,2001-09-16,false,1221300865.23,17:8:7.16 +1052142946,42337679778930096,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.375,1959-04-23,true,13456703.7191,2:2:11.34 +630607103,66626895706348024,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.375,1992-03-22,true,560200312.703,19:32:19.15 +-1334452899,27937267151105968,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.376,1991-10-27,true,1020986663.27,12:7:40.14 +2014673149,59638392015427432,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.376,1994-05-07,true,862017079.044,14:30:11.15 +954467047,54693842921111760,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.376,2011-08-16,true,674858151.415,5:44:28.20 +1792306716,67617325819267392,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.376,1994-05-05,false,47647680.1185,4:15:14.55 +-1283314388,47275856004972832,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.376,1996-03-22,true,580218991.912,12:54:30.2 +-74748916,26875968284129464,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.376,1983-06-24,false,616423892.903,9:5:23.53 +-858545894,82158320587879872,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.376,1954-04-04,true,840053437.866,10:26:30.22 +1682483848,65863180983677448,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.377,1952-10-11,false,4069428.79153,10:4:5.45 +-1757012774,55562105479353856,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.377,2007-07-02,true,94360552.2007,2:17:53.43 +-2020935230,57949782541371176,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.377,1964-08-22,false,430066738.536,21:24:47.25 +-397338462,8995578215002983,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.377,1955-09-05,true,980421917.484,20:46:34.56 +-1345759422,7697691284503480,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.377,1990-04-10,true,1115566874.77,11:48:12.6 +-137601553,35893319228992668,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.377,1957-05-18,true,519028400.32,2:25:19.3 +-1024530375,64989099422659432,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.378,1976-09-14,true,899001205.96,20:46:21.39 +-1979784972,49992938605037040,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.378,2013-05-09,true,478461813.11,20:24:25.21 +284131135,71303580333345272,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.378,1990-07-23,false,277157688.699,18:7:51.49 +708859036,25999947794414284,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.378,1957-03-12,false,7870334.19403,4:36:15.51 +496583060,42170323116729336,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.378,1953-01-20,true,455270894.204,22:56:12.1 +777487342,8972759432696689,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.378,2000-03-26,true,420592643.039,5:16:7.7 +831332397,74503857172445072,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.378,1970-03-24,false,427972069.954,15:25:39.13 +-802615850,47646300638948792,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.379,1986-09-07,false,124675997.792,10:31:29.39 +1452727174,43165135286373552,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.379,1983-11-18,false,641179215.624,12:41:38.47 +971897166,25263266835767912,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.379,2009-11-04,true,1282028370.93,1:48:29.2 +-1869264756,13339679603858104,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.379,1970-04-17,true,1308675707.48,7:26:13.46 +583794896,37486883368149968,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.379,1991-01-24,false,1114425952.88,5:1:51.7 +-700760834,33055256040983808,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.379,1992-02-22,true,818418464.991,3:3:35.8 +-1913523941,35104571580937820,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.379,2011-11-13,true,722103970.726,2:50:11.26 +-1033581770,67633137292159072,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.379,2012-02-06,true,550322123.861,9:50:4.3 +765858616,25521834189603708,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.380,1962-10-12,false,507807497.18,1:25:52.52 +1341927644,55541442375099728,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.380,2013-10-20,false,1339675921.5,11:26:6.52 +-858477429,45406534617898336,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.380,1961-03-12,true,1172141207.08,1:5:55.26 +-788157846,47017823825120472,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.380,1951-11-08,true,264038659.73,8:54:40.24 +1192099890,8469756046431795,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.380,1976-06-22,false,1370560272.52,4:28:3.42 +668558129,68608498265570344,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.380,2011-07-18,false,159959981.397,15:30:15.30 +903005697,35768082362252964,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.380,2005-06-13,true,1087241125.29,10:31:7.23 +437082969,85874078880561936,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.380,1961-10-12,true,334677623.458,14:22:24.43 +776854898,73494890805797280,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.381,2000-06-12,true,1114709321.37,6:38:2.57 +-1480611850,29131171166683456,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.381,1963-01-10,false,421083865.416,8:37:17.48 +-1849861507,23424425539137332,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.381,1995-10-26,true,221833414.637,13:46:45.26 +-1734445662,39125147459629760,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.381,1973-08-21,true,443315406.333,10:31:57.38 +-31489335,37958018467640344,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.381,1991-01-11,true,198398286.35,3:54:56.47 +-1561662245,89186756340251872,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.381,1961-11-27,true,430996281.236,14:18:36.36 +1519327848,21332574848350804,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.381,1995-09-03,true,851155633.463,3:12:55.11 +963824022,54140785223572592,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.381,1983-10-08,true,1229063024.5,13:46:8.8 +-2043123238,91057480654786496,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.381,1961-09-04,true,38853182.3034,1:2:58.57 +-259401283,76009522316266976,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.382,2011-03-26,false,1030853867.09,5:24:56.2 +473770201,50833587693508448,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.382,1987-11-12,false,20522570.076,21:58:25.44 +2007980723,66151942805868936,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.382,1980-03-06,true,1411499170.92,4:4:26.27 +33283973,24980255462875136,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.382,2000-09-09,false,40027969.4745,19:57:33.33 +-394870731,42465379909797808,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.382,2012-01-27,true,479921543.397,13:1:46.33 +1341503627,40740396186349040,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.382,1989-06-14,true,1170897379.48,20:45:51.14 +122231223,76971385344725936,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.382,1965-09-22,true,562285708.034,6:46:39.33 +897028496,44059875192745352,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.382,1951-11-05,true,584928294.414,12:11:9.31 +1126396871,34871734943326588,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.383,1974-08-22,true,1157039182.49,5:53:40.5 +-40376583,15915243582167890,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.383,2001-05-03,true,7558093.3015,10:25:3.44 +236839903,53712252391810496,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.383,1999-03-26,false,749918312.097,19:50:45.13 +1696880775,39453910870287432,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.383,1985-09-07,false,199595696.397,5:44:14.44 +-625124734,60453234400872320,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.383,1998-11-01,false,1240095005.86,22:32:56.22 +-1115556538,27899414113613456,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.384,1986-08-05,true,686376327.97,2:19:42.53 +948493434,42672527431711624,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.384,1973-10-12,true,594980269.777,11:15:11.1 +-416291828,38464415283589640,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.384,1988-09-08,true,565155733.802,14:11:10.57 +-314014880,22899555728423580,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.384,1979-06-23,false,1188596205.01,14:31:21.44 +1255031983,76215787035341232,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.384,1994-01-13,true,1169962331.04,7:56:56.2 +1814180912,33784308313632460,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.384,1975-02-15,true,1283748207.4,20:51:49.33 +-1899254713,40657763664010784,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.385,1999-06-12,true,521897706.387,22:25:13.3 +-2068795608,74430308994397520,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.385,1952-03-05,false,478218867.538,10:35:31.29 +-950432244,83321560601200000,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.385,1994-02-17,true,343072394.372,4:18:9.53 +-2040099309,39360340078428304,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.385,1968-10-18,false,665728770.046,17:26:52.10 +1595192765,33409822680858736,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.385,1992-10-20,false,1102824481.03,3:27:25.46 +1312875367,1326795053319936,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.385,1990-02-14,false,1139865327.19,1:4:25.36 +1503865048,17437471563497586,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.385,1987-01-19,true,1187745501.18,7:58:15.25 +-2062552302,34708644034620868,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.385,1974-03-01,false,699881049.938,13:16:48.33 +1094979040,44541063510515592,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.386,2005-04-12,false,231656499.937,1:54:16.39 +714276155,29401375795662264,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.386,1990-03-21,true,366266148.326,13:23:30.33 +867424977,14152557655330356,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.386,1957-08-13,true,860380648.725,8:18:17.47 +1943241953,30855648710423400,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.386,1975-08-22,true,920377000.979,22:56:37.29 +1486993872,5345816700446249,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.386,2006-11-13,true,1150953651.32,2:1:49.19 +-1883713961,1687340094253168,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.386,1970-05-02,true,265337641.068,8:15:14.51 +-1691547568,68419471589441120,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.386,2008-09-04,true,283697094.282,17:16:3.17 +-377315100,32132686578404312,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.386,2006-10-14,true,875451608.74,19:5:5.35 +-1034236435,16396071968448768,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.387,1973-11-01,true,513147718.272,7:46:31.10 +1870024255,74840357285199552,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.387,1962-05-17,false,685550516.327,8:42:37.57 +1355387314,62144262995275920,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.387,1967-11-02,false,1051092070.74,11:57:6.56 +-1062664863,38673220978729064,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.387,1964-02-15,false,229219499.217,15:34:15.7 +-2008177147,62959121969771920,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.387,2002-06-23,true,434994809.781,2:3:18.3 +-1918235698,66416118717992808,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.387,1990-05-09,false,1154982082.03,18:35:31.27 +-1043982510,35077357292089080,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.387,2013-02-26,true,512573285.416,17:56:35.10 +1068031570,23647642347012620,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.387,1969-05-01,false,81469556.2598,17:54:57.34 +782096794,16161567109894042,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.388,2010-06-23,true,634752345.223,21:52:20.39 +-1334225898,77353154133692720,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.388,1996-08-02,false,1404183939.85,16:50:7.15 +274935402,8139944808458291,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.388,1988-10-10,false,1358320437.62,14:26:13.39 +-376860124,22225143421833604,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.388,2004-02-05,false,1280817296.44,9:14:31.19 +743287795,65670130862232280,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.388,1967-08-17,true,162519631.255,9:3:41.10 +1820290234,71693027454040424,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.388,1982-10-26,true,710893566.039,19:43:39.25 +-1493360098,40775818800961792,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.388,1952-04-11,true,143394787.125,16:55:53.56 +-1501778856,83135724523093168,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.389,2012-03-25,false,434391282.716,12:5:47.46 +726064786,52837657170804480,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.389,1975-08-25,false,717192940.552,4:11:10.23 +-1035198634,88931635967990784,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.389,1969-02-18,true,1074990803.07,11:25:42.40 +1903440513,1309161237957027,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.389,1997-05-08,true,132919531.575,13:39:9.3 +-237053565,74450141338064656,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.389,1990-11-19,true,58596425.1716,22:31:30.25 +-1987572620,25009628350481656,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.389,1989-07-19,true,166714649.344,5:52:32.42 +703682518,86989172329925376,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.389,1990-09-19,false,1217817590.55,20:55:16.35 +1587875716,20165538185618648,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.390,1967-06-23,false,329027843.678,20:22:33.22 +-1707123468,83679048094410592,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.390,1968-02-15,false,396162069.508,18:56:2.11 +-1424165935,43366101080835648,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.390,1957-10-19,false,288310630.909,19:35:11.52 +-1793579945,85685366538467376,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.390,2009-09-20,true,520167954.852,8:54:47.2 +-1450159579,9215607626458450,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.390,1980-05-13,false,552370783.757,12:8:14.44 +-223564787,75874488275514640,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.390,1959-11-20,false,1300231483.15,20:20:28.21 +-1527878816,21937442261285624,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.390,2004-06-16,false,27817185.3385,12:54:49.18 +2053952953,61611177353346568,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.391,1988-05-18,false,1058180592.92,17:14:57.56 +-1210039314,86467080343892096,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.391,1997-09-17,false,658546473.157,10:20:2.9 +-1330396648,48424878310556528,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.391,1952-06-19,false,871927669.768,20:5:28.34 +-387505318,88677357003851168,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.391,1963-04-20,false,1333612022.28,21:29:1.34 +665311868,52841911651495240,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.391,2001-07-18,true,1382340093.79,6:11:40.24 +929624850,8434078855754957,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.391,1984-11-09,false,90532469.1983,4:39:39.20 +719168155,42350425825112344,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.392,1975-10-24,true,1353213987.98,4:28:6.27 +-1605649529,63642915769707152,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.392,1999-08-25,false,657598627.496,20:41:52.55 +-1755756432,62922052935490344,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.392,1981-06-16,true,830356593.81,15:51:19.57 +2100685942,67086708798737072,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.392,2006-03-12,false,1035910033.99,1:44:18.38 +-1912385369,37413472801566376,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.392,1992-04-21,true,904176561.222,6:35:21.17 +-418221895,54467724379089336,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.393,1961-07-22,false,105148152.5,13:6:43.57 +-719628948,32558865412298056,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.393,2012-02-22,false,308855619.077,8:57:35.30 +933977168,76349695214525232,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.393,1996-11-14,true,226091943.553,14:56:3.5 +-1394899876,34749481614643172,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.393,1979-08-16,true,468596593.317,9:41:24.2 +1507105872,36181769850529272,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.393,1963-02-10,false,1264561278.52,20:35:1.33 +-820296547,21053815769119060,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.394,2008-02-13,true,504844216.269,20:10:50.37 +-2042713833,47883029689913416,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.394,1992-07-07,false,84419243.91,15:19:19.41 +1433330942,61545483971389872,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.394,1959-06-10,false,651390711.726,3:23:41.58 +1599624204,89876469961779616,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.394,1980-02-07,false,940571965.422,9:55:50.30 +-1871293047,52120077680437840,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.394,1986-07-03,true,961005132.59,22:14:41.20 +-86299149,89932628112069168,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.394,1962-02-01,true,129506588.496,8:42:36.19 +1330552363,31424221618084844,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.395,1995-11-02,true,539722355.206,11:33:15.33 +1615943570,32817518048133336,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.395,1967-05-15,true,773215687.633,11:6:15.38 +-848161779,22292482396081980,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.395,1997-05-15,false,105564770.96,15:32:14.5 +-1305745441,54097135514775176,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.395,1963-08-20,false,895367882.54,22:34:57.43 +440348132,69734119915439400,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.395,1961-09-16,true,1416115640.52,17:51:38.24 +-1539123030,54479494369455976,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.395,1996-01-01,true,919923188.108,7:7:53.51 +1369619559,36660845993201144,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.396,1961-05-27,true,169055923.267,6:12:29.7 +-1526601506,8760979694384825,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.396,1971-09-23,false,137876359.242,5:4:8.35 +1996252944,46484983565195488,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.396,1991-08-24,false,263776848.808,17:56:30.42 +-793606504,61805136349226192,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.396,1967-05-25,true,355981052.441,15:25:49.29 +-1852007106,61927115700945512,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.396,1990-09-05,false,1104417.88625,14:11:57.40 +1857114985,39177245835464048,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.396,1978-04-12,false,1268815643.69,19:25:21.54 +-1945863915,50537952008713976,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.396,1958-02-17,true,1180058544.77,8:14:3.32 +1563949726,7161376892598067,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.397,1966-09-13,false,1071153138.25,22:40:25.24 +917558306,86303234805055648,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.397,1982-05-14,false,956480082.296,20:19:9.21 +-1946272082,17494038903351286,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.397,1976-05-18,false,993716670.83,14:49:8.45 +-127212925,82008144340514176,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.397,1965-02-06,true,1111872753.91,17:4:27.6 +-996505466,76222557314354352,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.397,1962-09-12,false,347430966.735,22:16:55.45 +1730747695,9598085129993626,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.397,2000-08-27,false,1421416923.5,10:50:56.39 +-1193219997,58625899575565440,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.397,1992-09-04,true,969387621.85,4:52:42.56 +1216893205,58274590692366592,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.397,2009-08-01,true,122011474.384,16:26:7.26 +1807466723,68766909737628744,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.398,1992-10-24,true,1320042859.25,16:47:35.50 +-996999083,90057446047064256,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.398,1980-02-12,false,348111870.535,7:57:24.1 +1947066395,21804712760898152,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.398,1968-06-25,false,889365231.447,19:16:51.31 +-631047480,39374602573973840,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.398,1971-04-17,true,1355001930.34,8:41:31.49 +-895888224,38743769069654928,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.398,1982-10-20,true,72882169.0003,17:34:12.40 +264116924,38006414000852800,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.398,1976-11-12,false,284462218.389,6:16:14.30 +328093541,8681899589523958,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.398,2008-08-11,true,502502884.711,20:55:26.38 +-1055888347,91164067871463168,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.399,1959-03-05,false,1201549014.26,20:41:44.45 +124403365,83678230467840016,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.399,1963-08-24,true,418490815.478,19:25:54.26 +1046196656,34475469227416964,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.399,1997-09-17,false,1429929856.44,22:8:17.32 +-1632364975,83502938042948112,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.399,1985-07-03,false,1430499985.7,21:49:15.51 +-149102516,45918081636486176,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.399,2010-04-18,true,61147861.7637,17:50:48.36 +-920140711,88467849082908480,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.399,1974-02-18,true,169663373.67,15:51:19.34 +41682043,33477584511699356,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.399,1967-11-14,true,369035159.905,15:41:19.39 +-1277039077,47727771303149008,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.400,1999-01-18,true,1029682419.67,17:53:34.35 +-1795480854,11157664606980004,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.400,1987-09-01,false,332600709.433,7:19:13.4 +-977980540,77259811318295248,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.400,1975-08-21,false,697427452.159,5:24:53.54 +719227468,21029219679686112,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.400,1994-11-01,false,727888490.861,20:49:27.16 +-1828995205,36950528842047112,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.401,1968-03-21,true,445464723.814,9:20:24.13 +-460849490,70290461255245864,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.401,2008-08-15,true,435189604.95,18:8:11.56 +-1633866900,49872266574167360,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.401,2006-01-05,false,696444388.005,4:21:30.44 +8463277,58624318520289816,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.401,1960-11-09,false,486878569.193,4:26:18.57 +-650872658,62767051483500672,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.401,1974-01-20,true,515442533.805,18:9:22.39 +-831741259,25002778849135676,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.401,1966-02-03,false,1121567299.27,13:34:54.1 +-1298257976,59803195498840176,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.401,1960-11-20,true,234264494.583,8:20:14.22 +1629741208,61740600855660320,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.402,1956-11-11,true,561071253.12,2:32:57.5 +886167256,69156445609410856,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.402,1967-11-02,false,873097065.527,15:27:28.56 +-895375082,55806712363914008,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.402,1968-06-25,true,794814254.355,2:30:9.7 +-1148867429,47410684039880848,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.402,2004-11-03,false,222075438.57,18:5:46.30 +-1457718693,39842567755445792,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.402,1989-04-19,true,838553880.822,8:56:17.46 +1321020984,89793315529642496,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.402,1977-09-08,false,721254826.621,12:23:54.32 +-1563431768,26022405846885788,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.402,1954-07-21,true,34470186.175,13:54:44.32 +-1114429995,11155878727327694,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.402,2001-03-13,false,854495877.606,14:51:56.48 +360395905,84095310925434800,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.403,1999-08-09,true,1045062692.24,13:9:15.30 +-112444357,51937068667033208,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.403,1994-09-24,false,662625686.707,4:44:19.30 +2019632406,51476844593305624,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.403,1992-01-23,true,862210353.93,17:40:56.29 +609107004,65291060437559320,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.403,1961-09-19,false,237340860.099,22:43:47.13 +1605887305,76150176698871824,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.403,1978-08-08,true,937128514.355,7:42:43.49 +-415969647,87564593841337904,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.403,2013-04-24,false,364149690.806,16:48:9.53 +170411250,8540990611304284,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.403,1955-01-18,true,391274506.678,3:30:3.25 +317101847,19375601777564848,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.404,1979-02-14,false,977832701.665,8:29:48.32 +1182907769,11077706949635974,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.404,1992-08-14,false,892735678.982,12:32:22.43 +861224676,47231040872283280,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.404,1972-02-26,true,795830635.691,21:37:58.39 +1814368965,7493097341037251,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.404,1954-06-11,false,295454448.836,12:42:30.36 +-1168136801,42019232929329576,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.404,1999-10-15,false,135871036.638,10:44:52.10 +11991199,2789234534032609,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.404,1952-07-20,false,788271176.884,20:15:24.34 +-2008816473,58870488831749896,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.405,2002-10-25,true,1715028.81648,2:53:19.30 +-881936911,57750227288499920,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.405,1963-03-10,true,450181625.182,4:26:14.35 +2080360999,78933137492470384,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.405,1957-08-23,true,58753066.8777,12:8:43.48 +-1126008213,44094639374870024,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.405,1955-06-11,true,532715395.87,15:4:35.41 +-657230561,76014787734355392,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.405,1957-03-06,true,396703891.081,12:55:8.57 +304673629,9822192676884152,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.405,1979-03-22,false,1069780728.22,4:4:10.25 +-252174425,33299886338816596,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.406,1963-10-09,false,68194416.0992,5:20:16.24 +1623253705,5483255305693143,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.406,1972-09-09,true,1288066107.28,10:17:3.16 +1438770826,79718188295624976,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.406,1995-01-05,true,616197076.242,3:1:27.29 +1217918383,35592209797260208,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.406,1988-04-20,false,941076694.62,22:56:51.35 +-827481743,10447126899475088,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.406,2009-04-25,true,786528601.637,1:13:33.39 +1143323642,62301467707371560,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.406,2013-06-25,false,119862901.08,6:25:9.3 +2100979597,22440817156629044,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.406,1999-06-26,false,859286584.658,16:1:14.25 +353821951,68448414208072304,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.407,1950-10-25,true,518770330.397,14:1:41.48 +-165012219,38631975704955352,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.407,1950-09-07,false,1188036229.93,13:35:57.8 +1164462889,36743431174111408,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.407,1973-10-18,true,8525040.61828,10:53:5.21 +-1324800328,45939567879450728,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.407,1975-01-03,false,1305697705.08,21:35:55.17 +-14731626,30134908690758504,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.407,2007-11-09,true,1081290948.97,10:53:5.53 +-1568945931,71382367637563976,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.407,1980-11-07,true,519516904.793,5:54:32.47 +1957979031,67871772039072224,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.407,1958-05-13,true,94930073.7965,5:51:45.11 +-1382912876,31156492027266672,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.408,1994-07-13,false,316170520.593,11:31:51.37 +-601264439,83754040533188560,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.408,1957-07-27,false,1201679137.97,14:50:54.34 +2074997200,81456119840066304,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.408,1966-08-27,true,1138365116.22,21:9:23.46 +1969280857,28538524798285548,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.408,2011-02-05,true,258968934.518,11:33:54.17 +-1226174955,73806282356051120,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.408,1973-05-24,true,480050565.242,8:19:11.3 +2016779454,75127152565607552,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.408,1963-09-17,true,571870052.538,10:4:25.28 +-883502921,45474630105842112,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.408,2008-04-03,true,161602821.864,8:7:27.55 +-1365092489,48195454187216120,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.409,2005-02-12,true,618083146.422,14:20:35.53 +-1709468966,58292901275344088,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.409,1987-02-24,true,1363147136.43,2:13:33.12 +976014520,14167825058863718,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.409,1961-05-08,true,412463226.845,9:44:24.23 +-1913962763,15757809530987428,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.409,2012-07-02,false,770056725.834,5:5:15.55 +1604010983,43365282969148280,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.409,1966-11-12,true,486896057.14,5:6:34.28 +-372685915,55906931500633232,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.410,2006-08-10,false,975198574.14,4:40:39.45 +-1276804035,43511711829873344,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.410,1971-09-09,false,140567128.95,12:30:37.56 +-1640663668,44583723659129256,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.410,1969-10-16,false,141020126.649,19:52:33.31 +-321441870,27416743084090060,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.410,2003-02-06,false,638691897.362,20:26:56.49 +616234766,32444680507217512,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.410,1982-03-17,true,1403279477.13,14:9:14.55 +-308399512,45623039100400160,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.410,1980-10-24,true,994543818.977,18:56:46.16 +955264549,21962940078917868,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.410,2000-08-27,false,1281937106.48,8:56:57.31 +-1438502110,18862998426173952,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.411,1999-01-09,true,1171853591.78,11:23:53.57 +-2089271057,16033659585408738,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.411,1984-01-11,false,23585661.3735,12:22:7.2 +-884171483,14268335811490264,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.411,1959-07-20,true,1401146589.06,5:17:9.3 +-455494162,23245030098302668,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.411,1963-01-07,false,811403530.931,9:41:18.44 +162450782,2299287260587591,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.411,2003-03-16,false,796560101.612,12:41:23.50 +1157272168,49935994332451240,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.411,1985-10-13,false,659805427.353,7:55:22.54 +142696348,7306748745700526,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.411,2005-09-02,false,940973970.809,21:54:30.1 +168915115,39109779392679600,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.411,1953-05-04,false,739348790.516,19:54:17.48 +-1310981673,74506460956981696,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.412,1993-10-02,true,1101437427.34,2:25:28.29 +-515594121,20628021816325028,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.412,1958-09-11,true,338907456.185,10:29:39.12 +-1146406343,87962328463897904,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.412,1996-01-24,true,532962210.509,6:58:58.51 +28841952,79352017674342512,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.412,1967-01-01,false,1269657512.5,3:50:53.54 +-179780284,85803894915570928,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.412,2001-07-22,false,1057402527.09,14:17:41.49 +-1771839728,48340862554870368,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.412,1999-11-22,false,115757561.882,10:18:18.44 +-1491236393,26422517796500380,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.412,2008-10-16,false,567530031.732,1:55:25.46 +-1917375621,66090664245441128,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.412,1964-04-10,false,1254937755.61,7:51:53.15 +1417257308,86308698692458592,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.413,1984-05-21,true,1136021247.31,21:23:52.16 +1964899930,53985054644798712,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.413,1989-08-06,true,305015071.475,17:7:37.9 +159400228,15598639629388288,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.413,1995-02-01,true,247911192.938,20:33:28.37 +-747928409,89841038793201376,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.413,2008-10-03,true,821516614.245,17:9:40.27 +844698303,27249084081467804,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.413,1987-11-07,false,40498590.1398,19:45:19.48 +2102429943,75692851941556160,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.413,1956-05-19,false,613198723.952,11:42:7.36 +1800285743,45387421356309000,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.413,1965-11-23,true,1074334554.93,3:12:11.21 +1987273038,3857505670212526,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.414,1959-06-15,false,210767041.739,7:2:44.47 +-955662307,40435270021078368,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.414,1975-02-17,false,692208192.52,19:34:31.52 +1224658287,12855888339390526,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.414,1989-10-01,false,206414583.592,22:34:19.13 +1004436505,32144308561580976,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.414,2007-07-25,false,727068686.337,21:12:34.12 +165643844,85450902708255648,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.414,1950-07-21,true,914411750.014,9:29:58.21 +-1798857431,40114541070199160,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.414,1969-05-15,true,1031907950.02,1:52:1.17 +-668494886,1819295548457963,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.415,1973-01-18,false,462909814.578,18:38:22.36 +1290663074,3615873220000829,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.415,1963-01-08,false,919464668.355,19:27:52.9 +-1516906726,8785229624103383,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.415,1970-08-20,false,680717184.021,4:12:23.12 +1695539484,69156169968488504,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.415,1986-11-03,true,830258594.66,4:51:32.56 +1550306990,59644046684377848,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.415,1962-03-20,true,1316346620.58,17:32:12.21 +-2022603899,34677497966247004,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.415,1963-06-15,false,859681800.992,1:40:58.21 +1524832960,83869018672357040,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.416,2011-02-22,false,1249928303.63,12:35:55.25 +-892063581,36054408125740592,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.416,2001-08-13,true,1379012490.06,11:22:37.36 +-448397384,13903475257897882,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.416,2005-01-13,false,1224705304.88,16:12:10.57 +-1271247129,10412186264273582,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.416,2012-02-06,true,767601821.719,17:52:23.32 +-151481448,24488324992462584,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.416,1989-02-14,false,1107547867.99,5:55:12.17 +241677034,2474323561791508,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.417,1998-05-10,true,1124250016.22,4:46:44.56 +-1369978898,20772640642878084,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.417,1960-05-17,true,1287147287.02,4:55:22.47 +-135321829,26377328016857660,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.417,2007-02-04,true,1193625237.99,13:9:7.52 +85091664,82163649908194064,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.417,1956-06-06,true,190059139.702,1:48:1.24 +780567647,65819622940766192,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.417,1977-06-26,true,596417616.418,17:20:15.12 +1433027438,23308291456712980,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.417,1958-11-15,true,1192410492.4,3:51:57.54 +-260998722,44440556925184240,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.417,2008-11-10,true,1141374215.51,19:38:13.40 +-913733329,54565178932151352,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.418,1953-01-19,true,319364413.84,12:31:37.7 +339019829,13206502106334750,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.418,1967-01-15,true,555443443.716,3:36:25.19 +2109814945,47206574808569344,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.418,1965-10-25,true,932407251.486,7:6:1.47 +-1081793899,10118800866944308,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.418,1998-07-01,true,843967705.712,12:6:55.29 +1216930900,13017078476944292,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.418,1982-10-24,false,515006043.219,5:26:14.37 +-1263974884,34962028513717904,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.419,1992-07-10,true,71604246.539,4:15:44.35 +643491039,29574301396042916,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.419,1985-11-11,true,987485889.713,3:7:13.16 +-178422473,1650478824188457,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.419,1967-07-16,false,14118689.0926,14:53:6.5 +-781028100,568331528944527,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.419,1957-08-14,true,759712218.943,11:27:1.3 +-1656043732,51072141273809432,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.420,1962-03-21,true,656748564.376,10:47:50.28 +1042778316,87769575430724416,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.420,1964-01-13,true,601724171.913,7:3:42.35 +1623192804,24258924033441640,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.420,1996-04-04,false,102017334.503,20:31:19.7 +1026793381,67838669797115064,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.420,1957-05-06,false,1301637753.38,1:21:37.38 +-915618192,18092454331407820,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.420,1958-10-19,true,301444715.291,3:37:49.31 +832963459,57952701322674488,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.420,1972-05-26,false,110041822.13,17:13:27.56 +2058816661,10577359231537940,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.421,2005-02-13,false,1319169171.91,9:14:16.28 +1124271756,57221463857848768,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.421,1965-07-03,true,163891029.44,1:10:8.16 +-373932531,30696992686891100,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.421,1969-01-04,true,454045062.033,20:33:7.3 +1750237891,69151011417340120,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.421,2005-09-01,true,655997859.411,5:42:40.5 +28285245,56497035923362480,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.421,1991-08-21,true,34193164.1161,2:11:25.39 +-162497149,33958780661187820,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.421,1967-07-02,true,604770362.436,8:32:49.29 +1059425896,27762003972715284,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.422,2000-07-02,false,688725353.376,2:21:9.51 +-2030608192,84336425526253504,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.422,1979-09-12,false,888303947.419,10:6:46.3 +-500987417,38879387629162240,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.422,1953-06-14,false,739963923.691,12:30:1.9 +560514637,12905262395436482,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.422,1993-10-26,false,1424266616.1,6:47:1.5 +-1550349272,66715936416286008,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.422,1985-07-01,false,633748925.776,21:50:4.17 +-267731334,52275818092466976,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.422,1983-01-15,true,596168401.079,6:22:9.43 +2051545552,21371260118863352,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.422,1996-10-23,true,350420917.031,15:14:26.58 +1195329736,10319128464991888,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.422,1963-01-14,true,886232517.246,13:27:47.27 +454233345,88706712931319696,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.423,2011-07-17,false,1417830803.4,10:15:33.40 +-421755419,45291790259333152,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.423,1952-07-15,true,38562498.2731,1:15:31.32 +1036516102,27383844365638924,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.423,1966-07-26,false,1146658790.52,21:55:30.44 +-363407350,39640936217366824,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.423,2001-11-15,true,1350476076.83,4:9:30.3 +1240458625,642317790413926,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.423,1990-08-24,false,1175870329.32,17:12:16.27 +363293208,35954057670624676,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.423,1985-05-10,false,415010423.562,16:58:54.37 +-32355071,87644525681112912,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.423,2011-03-14,true,677107273.948,16:53:33.10 +-1018713918,8650049463066491,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.423,2012-09-17,true,143685962.254,17:35:9.54 +393121822,64241675607797800,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.424,2000-05-15,true,386480051.132,7:56:10.38 +-503489992,40978519371841144,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.424,1961-04-13,true,343686557.061,8:5:47.28 +-2700128,42517258480679832,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.424,1965-07-15,false,916000603.406,5:46:6.44 +645418952,32117798326155440,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.424,1989-07-22,true,304307163.52,13:19:33.40 +-1545543566,68001144176838552,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.424,1987-03-17,false,84795699.7519,21:53:12.8 +-1643726251,51655758775294360,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.424,1968-03-08,true,955425188.385,22:10:27.47 +404559635,23177166334749552,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.424,1964-05-04,false,845817418.193,19:22:40.19 +-910080152,84339621740640544,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.424,1956-05-22,false,535849766.977,21:17:54.30 +-1961265436,39266665801309440,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.425,1999-10-19,true,1202548689.32,5:13:32.31 +-785929996,60487302678723336,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.425,1996-05-10,true,947413179.625,8:37:14.42 +682512012,5025671302195128,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.425,1979-03-27,true,259590652.782,17:47:25.56 +-855384123,58120046819677032,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.425,1964-05-09,false,1091276704.3,11:24:30.29 +-1201274416,11679783214614918,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.425,2004-01-26,false,794451036.815,3:35:46.1 +2042989269,44585361967730024,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.425,1973-09-04,true,357728451.973,10:5:33.31 +113408149,40542124135491552,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.425,1981-07-13,false,182156019.052,2:51:25.35 +1096382343,2242971953023303,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.426,1990-07-19,true,726913300.605,8:11:2.10 +1054147877,44343045372692984,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.426,1999-06-24,true,136395232.318,8:30:35.21 +-1196438873,38954830974247536,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.426,2007-03-10,true,1331662264.27,4:31:52.50 +139776358,72636310888166608,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.426,1976-07-10,true,359209736.237,21:24:23.16 +886189784,4077901978140354,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.426,1998-10-13,false,266134583.104,15:58:6.23 +1319060060,60257224548596376,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.426,2004-07-21,true,184558994.357,19:48:7.44 +929509349,55635854009308488,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.426,1990-02-05,false,10802749.3266,15:55:11.40 +-1875575259,57923590120503024,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.426,1953-10-05,false,659176694.419,20:11:50.30 +-698745827,80209623303523248,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.427,1974-03-15,true,1121224546.68,16:50:8.33 +567107136,34285769026231144,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.427,1984-08-03,false,698308206.406,6:25:49.26 +-1122768879,67793862382177680,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.427,1963-03-14,true,768253660.906,6:57:19.58 +1813544207,42975306719817656,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.427,1956-10-05,false,1407353893.0,17:22:16.6 +-730719251,79657356419466736,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.427,1957-09-21,true,314575792.905,14:54:24.27 +-154782352,26925094702992640,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.427,1988-04-07,false,293685667.233,12:3:39.37 +1000900624,91774119340984224,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.427,1977-11-21,true,439062873.932,19:35:40.8 +-1969352863,37290786486669040,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.427,2009-06-01,true,69946300.0279,17:3:14.56 +1594788576,34623539298254704,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.428,1991-06-21,false,379395634.11,21:49:3.11 +-1103532560,2026191826513756,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.428,1994-08-25,false,1319577260.84,7:43:8.45 +1689581580,33828567757016280,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.428,1993-02-08,false,1368316330.54,6:8:42.43 +498521772,76741615296282896,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.428,1950-06-17,false,630409792.333,3:34:34.37 +124019155,36565295355968072,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.428,1955-08-16,false,686570141.613,13:14:28.45 +1374620824,11995809350761318,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.428,2012-07-01,false,888652067.025,22:41:14.15 +610626243,46084423005799528,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.428,1955-10-17,true,7556278.49793,18:21:28.24 +-426016347,54928769720248656,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.429,1965-05-16,true,140432861.509,3:4:25.56 +-1910210985,84958987921462096,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.429,1980-06-22,false,367759371.242,22:28:1.46 +41929406,40152579910916200,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.429,1959-09-20,true,225971945.299,13:55:43.44 +-35021274,75681556173697920,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.429,2005-02-14,true,1122155656.99,21:40:35.20 +-1456957399,64541359704757048,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.429,1958-09-15,false,432151870.78,3:33:2.18 +1922487350,38999105172941992,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.429,1985-09-10,true,315613323.228,19:5:7.11 +212291815,51448183675676936,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.429,2004-11-04,true,516501568.461,14:55:51.17 +-1469513692,59005722916320984,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.429,1986-11-23,true,1162618328.72,19:18:10.55 +2001375283,37901345701277488,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.430,1978-04-19,true,299886062.651,1:9:49.27 +-1134375977,75363481861227104,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.430,1985-03-03,true,1061610093.48,6:43:20.25 +161652215,40200750125469536,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.430,1991-05-21,true,630503595.684,7:7:17.15 +-283325400,42558346666128024,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.430,1978-06-26,true,1243300575.87,9:9:50.56 +707691048,7535244080430766,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.430,1982-11-10,true,1252994806.66,7:25:56.35 +-11099115,53533801441117432,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.430,2008-04-10,false,22050812.4713,6:36:10.12 +307370772,4519016051828920,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.430,1974-11-01,true,1380863026.76,21:41:24.46 +-1814467339,82364048401354912,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.430,1951-05-23,false,157616028.626,6:4:30.26 +-905605344,40005898945354960,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.431,1978-01-24,false,835733805.312,16:36:6.38 +-68701700,54678528433845648,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.431,1963-06-07,false,508174647.291,15:35:16.18 +-1826858097,84304479260777440,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.431,1964-10-13,true,332838986.21,19:44:2.24 +-924377249,7903321378438380,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.431,1975-06-03,false,169484259.796,11:51:57.51 +-1024951322,75671109568827360,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.431,1955-05-06,true,512562713.736,17:31:1.39 +-585419401,82361592291392368,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.431,1992-06-26,false,145001039.492,13:18:31.5 +-361416,37931915813950448,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.431,2001-01-16,true,765612594.021,20:16:21.40 +-729144835,55205183826022440,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.432,1964-10-04,true,1103684910.13,16:34:9.55 +952526744,34296953908218132,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.432,1967-10-21,false,101070579.499,3:52:48.25 +-1963024866,55260713705475096,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.432,1995-11-06,true,572910899.955,1:46:42.20 +-534754500,88535382307368736,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.432,2005-06-26,false,770181356.082,13:16:10.49 +1294357230,42523717053154448,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.432,2001-10-07,true,1412979931.07,13:56:26.10 +-1452314865,51396392182351344,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.432,1974-10-27,false,20935808.0182,18:37:49.24 +685218448,4461305021890611,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.432,1968-04-06,true,1229486133.36,13:15:14.47 +-1738502616,38961760868572744,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.433,1966-08-12,true,1396654709.28,9:29:41.28 +-164105106,67443812936183904,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.433,1969-02-13,false,879675556.487,22:23:36.45 +1187616829,44723008569618496,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.433,1982-05-12,false,1275007994.22,8:26:20.43 +-1441877843,20253528456938240,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.433,2004-01-11,true,1129839269.38,22:18:46.56 +-1583488796,24530740993255344,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.433,1998-09-06,false,252881553.036,15:4:2.53 +-742689004,22982053003368252,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.434,1958-08-04,true,646582134.963,10:9:47.49 +-244792197,19589750264447540,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.434,2005-04-21,true,929578256.294,21:29:19.28 +-1683295158,62092449246851176,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.434,1992-08-24,true,683310929.104,9:39:52.31 +878420031,58848219211800424,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.434,1956-08-06,true,1422148159.17,2:13:31.5 +1179896511,72504884772992864,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.434,1975-02-09,true,882235229.18,14:46:10.31 +1870966295,28941047782221220,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.434,1971-06-03,true,1067121236.79,12:46:49.4 +-686144840,86168335469624592,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.435,1975-05-02,false,763380005.833,4:7:30.53 +-1699751512,1935266776647393,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.435,1976-02-03,false,1285576609.89,19:29:41.16 +-2071533998,73754813578681584,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.435,1984-07-24,true,518490831.382,3:20:49.41 +1332745152,22459031841598220,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.435,1976-04-17,false,1258391008.64,18:31:30.25 +-1591531732,16316975472805162,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.435,1958-07-24,true,934869655.025,10:16:15.21 +1316009980,81530617687870576,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.435,1956-07-17,true,742839185.354,12:8:45.44 +-1195717101,33295421229169348,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.435,2001-04-20,true,113839950.218,15:10:5.39 +-1607664890,75440115937344560,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.435,1957-04-08,false,74235310.3697,3:51:5.54 +-1004438035,90474407510644592,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.435,1990-11-24,true,826207986.272,8:35:2.33 +-1533357518,49150994829849976,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.436,1961-08-27,false,468128748.566,11:49:20.16 +-1321824043,79100145587256192,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.436,1951-02-09,false,803297304.725,12:49:4.49 +2062728481,66984915164890168,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.436,1999-08-13,true,702447094.939,4:23:11.4 +714027462,28186093229992304,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.436,2001-03-27,false,1362342883.7,16:25:16.19 +1991475198,59848478115225008,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.436,1962-06-12,true,942654755.704,3:16:39.58 +123840845,22755375324901316,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.436,1968-06-25,false,1272043429.23,2:47:51.10 +1263226317,90737705165233632,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.436,1956-05-01,true,937634808.338,3:33:29.48 +2137100342,32792335365635668,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.437,1970-07-15,false,1329873311.7,15:39:37.5 +2133451757,89091925730737152,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.437,1972-09-15,true,345460780.039,21:33:53.16 +2132823006,72922310425874416,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.437,1991-07-09,false,1280411450.08,15:9:5.51 +245287177,74329827612501360,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.437,1991-04-06,false,211721040.961,4:23:46.51 +2069415733,79832934459179456,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.437,1958-07-18,false,1285694375.82,15:38:22.27 +-1248772424,26321208531589532,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.437,1963-07-22,true,1123191234.54,17:27:19.31 +-1626710376,13729452917378766,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.437,1995-07-13,true,1244517684.93,18:52:43.6 +-1684483898,24432994757283740,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.437,1950-09-23,false,1406445136.68,20:36:36.25 +-1454779969,58239443844727512,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.438,1981-11-16,true,234335111.595,2:9:55.32 +-1847867541,85581520823316576,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.438,1998-03-20,false,969325391.882,16:30:39.22 +-690200199,60888208293323912,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.438,1979-09-01,false,139988717.53,12:54:28.24 +-905544690,11059080500682854,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.438,1967-06-11,true,578449309.22,10:48:39.22 +342059992,82480211247804576,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.438,1955-09-12,false,89915462.6363,3:20:32.55 +-1376414483,44150604613180848,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.438,1995-10-09,false,1387388494.08,15:19:18.44 +396592745,61447658923205288,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.438,1974-07-17,true,1066264237.41,19:28:27.30 +-6865742,57453244371983024,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.439,1980-06-06,true,1027568486.36,16:57:46.38 +-1497647337,11846093754127902,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.439,2000-11-20,false,1225333918.19,22:23:51.17 +-225509280,71065945473832656,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.439,1957-08-14,false,1085646055.63,17:29:24.8 +-1461257468,11363554059139626,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.439,1992-04-27,true,505709149.966,10:20:44.50 +1172403178,58960021146440128,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.439,1976-02-14,false,834755048.5,17:54:33.1 +936592391,14588430705758434,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.439,1953-08-15,false,674134872.243,15:45:5.5 +286880137,31931396830284024,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.439,1993-02-25,false,376365528.778,6:29:32.42 +1796033282,45572981070911072,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.440,1991-04-01,false,1063110591.17,15:56:28.49 +-1176226636,70567608181517880,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.440,1961-06-15,true,1050553788.43,4:46:50.3 +85789401,44467195704237064,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.440,2008-02-10,true,425303056.784,17:54:47.27 +135017247,1079269942695198,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.440,2002-03-08,false,123862929.996,5:22:46.47 +155573954,33238260044862792,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.440,2002-09-01,false,981682832.881,3:9:42.41 +-927389838,83940421650376576,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.440,2008-03-08,true,134939691.112,5:40:21.5 +-323490541,91542768003589536,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.440,1999-08-11,true,1027973853.97,12:2:20.42 +-130952350,57454795187473928,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.441,1958-07-26,true,25228526.0542,2:38:15.3 +-495667751,77446662051086752,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.441,1974-08-13,false,660986824.701,9:31:9.56 +1675754351,29738314285156936,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.441,1963-08-23,true,1038201029.15,6:13:51.30 +1439445636,5467062729240812,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.441,1994-01-22,false,1252685230.87,22:25:47.53 +1357964654,85417425959725472,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.441,1985-10-11,false,267647384.303,11:22:8.35 +1191113055,67976366641156920,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.441,1997-03-16,true,69030024.0901,18:37:3.43 +471553173,88672689407337728,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.441,2008-08-25,false,511497090.949,22:3:33.2 +650849974,83266908302752384,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.442,1963-02-11,false,23518866.3702,16:38:50.24 +1977909793,5651723735074785,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.442,2000-08-17,false,466803488.732,22:11:33.40 +-1498403115,28163306958275440,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.442,1995-10-10,true,625651577.232,11:22:8.45 +232804917,29446423755603508,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.442,1984-07-01,true,77663507.7456,16:4:49.30 +1039740048,15787921379360124,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.442,1985-04-06,true,1039644427.71,14:42:57.1 +1284588683,74467568598288256,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.442,1962-07-03,false,1239164750.77,19:58:9.11 +514839557,19649925513100300,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.442,2011-09-09,true,943116252.304,19:4:56.47 +1874208258,22530797789322812,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.442,1963-11-19,false,1167533690.12,2:4:18.32 +525606661,65677313419151432,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.443,2003-02-17,false,1035940822.55,6:46:51.57 +-1942233607,7297394195045857,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.443,1994-08-10,true,1034615710.38,19:46:7.39 +206658802,70608947786122320,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.443,1964-03-07,false,1416959313.07,22:11:53.22 +332163755,74777883695190624,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.443,1974-08-06,true,874453644.617,1:48:26.26 +2114076357,7086441051438889,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.443,1953-02-22,true,511528105.314,21:5:29.42 +2137266948,383930934094561,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.443,1966-01-24,false,138805715.609,3:27:42.49 +746097174,58346879919566648,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.444,1984-04-23,false,482493169.163,14:57:3.55 +-2034494104,38077412861071992,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.444,1976-10-08,false,811493970.152,8:24:56.39 +-1798026463,48084065631145472,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.444,2013-04-27,false,609661048.511,10:51:16.15 +1938174406,6223874673603707,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.444,2009-10-24,true,308837647.771,16:20:19.55 +1418361505,17597779353801062,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.444,1961-08-26,true,524531815.847,5:26:19.22 +-80023037,33095919802173992,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.444,1955-05-24,false,1226177651.51,19:18:49.1 +1758556775,25308642185322344,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.445,1954-08-04,true,836173815.358,17:45:42.30 +233326448,64461815636299432,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.445,1959-10-21,true,280649951.015,16:56:4.4 +-1843026857,1368068151743242,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.445,1954-11-12,false,1407645354.25,22:10:25.28 +1373176432,9470714059472660,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.445,1959-05-04,false,914259153.543,1:37:24.49 +600051635,40888161031370600,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.445,1999-02-27,false,985984587.393,18:53:22.5 +1010964199,36800210197295296,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.445,1957-03-01,false,999330616.695,1:48:42.37 +-1488779423,71014254355094272,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.445,1991-02-26,true,1218295717.42,4:10:41.2 +-172317984,86786203524687840,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.446,1995-09-09,false,455061427.071,17:56:27.58 +1385163525,53206540363791576,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.446,1996-05-03,true,1423906385.66,12:43:49.55 +1778250106,10606240438219970,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.446,1990-06-19,true,774097901.567,18:30:12.37 +-1015762700,50943067540064376,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.446,1969-11-19,true,631141373.642,20:55:42.9 +1985636624,32270541180263220,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.446,2006-07-12,true,1221336056.78,22:18:40.1 +-814078867,32818266460860068,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.446,2011-05-03,true,1336998886.99,3:42:11.38 +1828470505,74803922314047184,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.446,1998-01-02,false,1343467396.03,18:42:47.15 +-1142827612,25172127819722988,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.447,2012-11-25,true,1062615618.17,4:7:5.52 +1914381938,61351976108967384,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.447,1954-06-23,true,1017707875.24,12:55:53.24 +-1005991098,53064888179293112,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.447,1966-03-06,true,460630841.67,19:45:54.31 +-376339094,41832670041623152,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.447,1984-06-06,true,1254297449.89,16:4:58.3 +1400914911,44592154302017280,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.447,2011-01-18,true,819475791.294,13:55:35.18 +1473236347,5141741501901138,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.447,1989-04-12,true,688707668.088,19:8:53.49 +-1723183786,5413458765107507,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.447,1998-11-13,false,454937366.959,1:7:6.45 +618184878,63835355662220608,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.448,2000-11-25,true,161268006.746,2:34:14.53 +-966822592,15840041791660534,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.448,1957-03-05,false,301885312.159,8:22:29.55 +-436730344,63819934416878360,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.448,1982-04-21,false,784544065.766,22:31:14.38 +-1529589616,39589468414400096,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.448,1999-09-02,false,127992290.812,7:25:26.21 +-1030341815,86424599767561728,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.448,1968-11-20,true,572442349.522,7:43:38.4 +-2005654327,54819339692584528,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.448,1971-02-25,true,586658671.736,3:21:1.24 +588522207,6985550903415265,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.448,1960-01-20,true,480535947.351,1:57:26.11 +-1586875513,69506505853489816,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.449,1954-01-05,true,36858871.6159,2:32:45.48 +1641395964,33222186452659752,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.449,2010-02-10,true,578179080.109,18:16:33.45 +-991544769,49331362962538104,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.449,1993-09-26,false,730276830.731,17:53:25.2 +1264595485,75957637495543264,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.449,1950-03-24,false,662898972.132,1:13:16.43 +-382712329,14717116235743140,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.449,1986-01-04,false,378905505.588,22:18:5.19 +464582513,15709732059106080,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.449,1976-11-03,true,1371919783.59,21:27:17.20 +-559849270,61192004698848664,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.449,1970-05-17,false,1189766484.42,15:49:44.16 +-1074187860,47521306408603864,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.450,1984-07-05,true,131333452.921,21:43:45.57 +953989295,70335223729221992,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.450,1977-01-02,false,668105432.282,17:35:3.35 +-223705122,63899651940565648,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.450,1964-05-03,false,1336431949.76,16:23:54.43 +964266523,719012952385413,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.450,1993-05-18,false,1382002790.66,16:16:23.58 +66862950,31105849709610232,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.451,1994-04-12,true,449075188.892,3:45:38.58 +-79563952,63892641153838232,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.451,1968-10-14,true,1377468803.9,6:22:26.6 +-178092684,78747474587003088,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.451,1998-03-11,true,261016972.544,6:1:26.34 +380713499,81339221327104848,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.451,1956-11-01,false,652933812.353,14:51:40.30 +25774104,90655829399852912,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.451,2003-03-10,false,417561063.714,20:44:49.50 +-2138921754,33266681382838836,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.451,1984-08-27,false,1100055056.0,15:28:29.4 +-383673870,68779221355711624,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.452,1950-07-13,true,874188359.778,17:24:22.6 +2059961551,27651161538802720,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.452,1973-03-01,true,811614461.109,9:57:32.14 +1963286845,13658566703645686,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.452,1959-11-01,false,467619942.983,9:30:21.5 +1340099393,90308482781147696,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.452,1996-02-04,false,1213342455.67,17:51:32.23 +-1300219873,85187871041381120,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.452,1979-03-20,false,863768150.124,22:54:25.14 +2123865220,38536086284878336,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.453,2006-05-24,false,172202698.204,19:55:18.33 +858196562,36634189171148680,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.453,1986-07-20,true,326114753.495,20:2:44.38 +317966431,77528910934957696,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.453,2010-10-02,false,719919974.324,2:44:53.40 +-302838600,90421005882152912,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.453,1968-11-27,true,1257400906.54,20:25:12.35 +-1851384273,30441706955949292,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.453,1987-02-27,true,1088049221.35,11:5:4.32 +-917907606,23982425008041544,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.453,1970-03-24,true,869363050.081,7:2:57.24 +1820887267,35711981139105620,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.453,1951-10-13,false,611978260.777,16:32:5.28 +3056476,57441628250767256,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.454,1956-08-27,true,1169168839.03,7:49:21.56 +2034133458,61149637620513672,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.454,1979-06-08,true,1101354969.38,19:58:11.31 +-1159354490,86954282616048176,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.454,1979-11-03,true,874528174.253,22:31:22.52 +1040253624,90266307763236704,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.454,1996-04-07,true,632081300.37,22:55:7.52 +1880176495,16214561036123320,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.454,1976-09-02,true,1017601078.51,2:45:56.20 +-1203005302,53254408351114560,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.454,1997-11-19,false,694736308.441,15:48:24.34 +1888273160,81453474855270624,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.455,2000-03-15,false,312649405.985,17:29:25.15 +25465255,92005506396867008,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.455,1995-06-05,true,760092247.746,5:39:42.44 +-1878256281,80831802513377552,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.455,1976-03-10,true,418704126.091,15:43:45.5 +-1790519179,69893935836020360,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.455,1961-01-01,false,1306001979.0,18:29:4.3 +54036205,7614177876829450,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.455,2010-03-01,false,1304591434.13,20:47:41.9 +2026459585,88768785213949408,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.456,1982-02-15,true,1055616155.39,5:48:12.16 +-380331099,69033691074730320,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.456,1999-08-17,false,331278180.521,11:57:53.16 +1828281796,1111142717774704,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.456,1974-06-09,false,157730839.515,21:16:16.12 +834624574,32836487102975968,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.456,1977-09-11,true,424503501.6,18:56:56.42 +-2061489982,59732709982768896,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.456,1993-05-03,true,887972323.119,17:37:34.14 +-1750328484,17201545381043150,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.457,2013-04-19,true,2248908.86017,21:52:26.13 +-1913557781,45477164678926648,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.457,2010-10-27,false,274902733.2,11:36:51.17 +1616135936,91904590457567312,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.457,1988-05-08,true,63779989.6003,19:10:40.22 +-1262727008,18454938506290576,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.457,1994-02-06,false,1055367231.51,22:18:16.24 +52903794,73931757233334512,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.457,1989-04-09,false,625683243.799,20:27:41.58 +-580169676,50638317949544400,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.457,1978-11-16,false,700366569.611,6:31:44.22 +1203711821,76602577308751840,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.458,1966-07-04,false,855712750.986,16:55:31.55 +-977688074,21924681784899552,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.458,1960-03-22,false,980946203.862,3:10:55.29 +1378322621,27364002756568944,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.458,2005-01-20,false,849544441.776,15:26:24.1 +922594695,66913929223886448,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.458,1993-03-10,false,720401505.231,13:50:6.56 +139518510,2088819556895467,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.458,1997-07-18,true,326817053.375,6:49:44.56 +1848675577,19544268399107440,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.458,2010-08-11,false,361109078.307,1:35:14.37 +566634326,77242820220885552,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.458,1953-02-12,false,610827191.902,14:40:54.1 +731533588,5201282734784277,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.459,1955-04-13,true,670224074.567,7:29:29.13 +1894765014,33990978456890716,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.459,1980-06-07,false,823029423.136,1:5:15.18 +-1661615237,24958553570453404,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.459,1984-11-27,false,515244816.714,9:56:19.28 +588296456,78417070376627056,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.459,1995-09-01,false,1138401400.68,14:49:53.30 +494845119,11696595555909632,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.459,1990-04-18,true,729815900.849,13:1:38.7 +1219234715,38688498586900760,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.459,1978-01-22,true,629653989.594,12:16:58.12 +-469220610,69957663692193864,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.459,2013-02-04,false,41614748.8417,17:40:47.56 +-125636461,36535353785753408,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.459,1988-05-26,false,274811017.929,12:9:13.37 +-1256484330,11887136880642776,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.460,1995-02-03,false,763654303.021,7:35:25.34 +-1905884225,51272757000589688,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.460,2009-11-13,true,44515283.5032,5:10:27.35 +1957577335,24484694487672176,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.460,1997-03-07,false,455071571.01,21:14:9.56 +1173253179,27778325160910100,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.460,1988-06-19,true,183553556.804,22:8:31.42 +1135121453,5161927738059264,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.460,1976-01-18,false,297085986.163,15:30:52.10 +1390472226,18774680369159608,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.460,1981-07-24,false,1380882660.55,21:33:7.38 +2021808390,79134919337490144,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.460,1978-10-22,false,1110353424.39,2:33:15.56 +-1645023783,81854720911928096,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.461,1989-09-03,false,1033910973.28,5:28:23.10 +-698488769,38140882369010384,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.461,2000-03-18,true,898173886.756,19:27:22.34 +998726565,30017288015631892,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.461,1993-03-19,true,637570988.501,13:27:22.51 +-523527282,66256438219888040,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.461,1995-05-02,false,355984303.65,15:48:55.37 +-1231085706,9554076169848606,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.461,2012-11-16,true,465320681.564,11:40:42.29 +-8226125,46878297657255568,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.461,1986-08-23,true,1277610391.25,14:36:36.25 +-36249515,86377083395376048,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.461,1974-08-18,true,29778034.8827,10:2:16.34 +-1748886332,58864165160319000,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.461,1958-03-14,true,694866344.122,7:26:50.46 +571490288,52602830461712016,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.462,1976-03-12,false,874331163.47,9:28:21.21 +-729550125,22730554085407272,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.462,1994-05-06,false,547886155.198,2:10:32.1 +1549273027,12207031094897900,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.462,1980-05-12,true,1126679954.42,13:8:44.26 +592689160,32560490510217300,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.462,2002-07-17,true,715368124.221,19:2:21.57 +1211467821,6880766326032292,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.462,1978-04-09,true,596279415.833,10:34:11.39 +-17993093,67017711640515544,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.462,1972-08-17,true,4008178.67308,3:33:21.42 +1364008332,11712137972272384,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.462,2003-04-06,false,1120989091.28,3:36:23.55 +1905631388,89136912750775344,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.462,2013-05-22,true,100867900.995,15:38:14.58 +-1404004078,35943753942780548,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.462,1961-09-06,true,314294873.589,20:2:4.37 +1992367425,35806700790256036,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.463,1965-08-14,true,12600121.2804,10:6:50.34 +600003265,71193878632898952,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.463,2002-03-11,true,825048004.463,8:38:18.37 +-690661825,76017736599548352,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.463,1961-07-23,true,1141336107.19,9:10:25.48 +-194189341,79527644760324640,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.463,1962-08-20,true,1168881112.93,7:19:27.32 +154946653,68078786214668400,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.463,1959-04-02,false,1400201327.76,20:27:23.49 +272102996,4350819611347722,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.463,1969-01-27,false,480953080.618,14:2:38.27 +-273368131,23900270627132896,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.463,2004-09-21,true,373018244.012,20:10:35.48 +-1239804828,56797538128889640,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.464,2012-09-10,true,1401031363.13,20:1:27.5 +1661574166,21972146477919388,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.464,1972-07-13,false,1022355330.14,20:32:29.29 +461554184,45758022776890296,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.464,2013-02-14,true,1198767815.35,22:20:7.3 +-1164380617,66511075318385464,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.464,1991-06-06,false,1035711880.18,19:1:13.46 +1923200375,44925811037918568,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.464,2007-04-13,false,248800451.165,13:33:1.40 +1204504975,73228874685627296,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.464,1967-04-08,false,77868563.2978,18:13:47.13 +-1951946955,13865753190669866,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.464,1964-03-17,false,944274294.613,1:58:36.14 +-1593624192,16133991748718838,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.464,1951-11-17,false,1032019391.01,15:29:25.40 +-1777114497,41371825462504856,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.465,1987-03-02,false,205342724.869,21:51:35.27 +-236529418,50931685482996176,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.465,1976-10-03,true,206861488.609,11:14:5.24 +2098472882,38084643250513808,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.465,1975-10-18,true,287198111.987,7:13:24.11 +1329854921,8293087277180959,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.465,2011-10-08,true,432643025.559,1:50:4.5 +2081744486,49012180596447480,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.465,1995-10-27,false,757122160.92,14:53:4.3 +1151602723,84499716856369632,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.465,1981-08-14,true,209126570.565,3:13:43.24 +890509428,77735804040661360,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.465,2013-08-01,false,330449515.706,21:36:1.33 +2136871459,40527461672836320,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.465,1982-04-09,true,1021615311.64,4:52:40.40 +-1209069163,22458946276909252,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.465,1981-10-03,false,609182639.043,15:2:48.11 +-1874279006,9841569814240646,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.466,1968-08-20,true,873626102.025,14:24:46.57 +-1529684603,45419179153620304,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.466,1975-01-02,false,418846936.314,15:8:38.5 +-468530052,58216601488748680,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.466,2012-03-05,true,781971548.282,10:45:20.9 +156933798,23655159064616296,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.466,1992-04-24,false,1042802391.02,11:10:27.56 +-133966178,85101050901228144,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.466,1967-04-15,true,445669433.466,11:36:14.45 +-2062483316,23609946541605204,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.466,2005-07-10,true,1168252910.47,1:7:5.16 +-1772905369,87063055470106304,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.467,1979-08-21,true,342103959.492,10:27:12.2 +2024800250,62937876641035648,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.467,1989-10-08,false,74134248.3031,11:36:14.25 +-1397862779,56592717085306224,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.467,1972-06-13,true,1065012254.66,2:10:48.44 +315781826,79388298308947872,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.467,1966-01-22,true,129283530.725,7:1:53.1 +1929428565,63362817303426384,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.467,1963-10-03,true,355359239.874,16:5:17.30 +169334128,90580544241567392,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.468,2004-08-13,false,454418240.655,14:17:7.16 +430320402,84015921869392752,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.468,1989-02-07,false,1204897977.58,11:30:2.28 +1468669488,59869589516651368,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.468,1951-09-11,true,300575222.52,7:52:2.33 +62089444,89376006471148672,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.468,1965-03-09,true,1127501245.73,17:19:57.45 +441030775,61292934863717256,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.468,2010-09-23,false,544784257.356,1:19:48.51 +1015709107,18823432862887272,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.468,2011-05-23,false,1335689236.42,21:38:34.14 +778956753,57943868792240664,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.468,1963-05-01,false,137113990.986,8:13:39.57 +-808602752,65461988242024632,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.468,1994-05-22,false,1047913648.64,15:56:31.32 +-1956718574,3401301478579097,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.469,1981-09-11,true,1093471624.47,7:58:3.29 +1016372964,68401831217772712,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.469,2009-10-17,true,267879284.901,20:41:44.58 +433409732,24816180013475400,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.469,1997-05-22,false,948844925.908,16:47:56.44 +676778145,40737695015563744,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.469,2009-09-13,false,62459678.4908,20:38:57.12 +-1119650354,66691990308011880,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.469,2007-02-22,false,1164883088.3,3:28:5.11 +431252025,29067157422723944,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.469,1992-04-24,true,998515612.891,3:31:42.19 +2115988023,37817919173217200,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.469,1975-07-12,true,750002000.99,12:47:34.4 +666946440,14613436611634996,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.469,1966-02-25,true,181577783.165,17:1:57.33 +2143311874,28478902279966640,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.469,1986-01-02,false,1185896124.44,6:44:13.33 +-1058494691,53779723061109112,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.470,1972-09-19,true,3782145.28608,15:2:57.15 +1465992037,51075882192483736,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.470,1965-02-21,false,1238633493.4,10:28:41.30 +-484146801,20427832778202524,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.470,1961-04-26,true,1170036511.36,15:41:26.34 +-2016252179,20306157398100704,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.470,1989-05-18,false,539642527.186,16:37:49.54 +746151029,89934777669412256,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.470,1974-10-24,false,607828192.18,4:50:7.5 +-1114021190,47720080872854168,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.470,1988-03-04,true,180410052.699,16:22:30.39 +1002033844,22841224984560784,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.470,1977-11-04,true,818639597.01,4:5:43.13 +-2076240885,25651365999984588,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.471,1952-10-16,false,1284340189.32,10:18:7.13 +1859100035,43499651264995056,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.471,1993-01-03,false,1143234348.97,10:53:37.31 +-301272414,36590920852147696,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.471,1970-11-15,true,1321493163.66,22:37:33.57 +1641972646,88772362830402272,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.471,1995-01-08,true,656421764.604,2:6:48.19 +1620255682,24932888565031108,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.471,1982-10-05,false,984729441.768,2:26:12.37 +1351422920,66750388954649168,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.471,1984-02-04,true,193030385.88,3:19:39.31 +1680280972,31425008330331924,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.471,1967-11-20,true,1364394352.94,8:5:46.54 +537095734,1871705206548449,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.471,2013-08-08,true,810771095.404,11:58:45.19 +-2011230743,42286531761913128,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.472,1975-01-16,false,739674666.497,10:15:15.27 +272129819,75670352763572752,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.472,1977-11-08,true,450342646.105,1:23:55.25 +569220897,63796849260978824,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.472,1958-03-15,true,990746361.889,10:41:43.31 +626553951,72798598575064880,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.472,1985-08-04,true,328955483.285,1:58:12.27 +-1791032370,82411510748563264,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.472,1950-10-27,true,127139386.307,13:13:17.27 +270837990,89586698992649776,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.472,2010-10-25,false,1206360606.57,12:10:1.55 +-1959250208,27337845739579608,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.472,2008-02-21,true,160639597.658,21:39:18.37 +955312748,18281708447551664,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.473,2004-10-05,false,598145535.8,1:32:11.18 +916132224,7899206071574170,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.473,1991-06-11,false,277324413.106,16:46:19.42 +-795709652,18716094577008272,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.473,1971-02-04,true,449535576.679,20:16:15.55 +-669163424,61062213450543904,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.473,1956-05-13,true,305858381.069,4:9:36.32 +518053233,34046137152659304,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.473,1976-04-25,true,1298841629.29,22:2:54.5 +-1022771302,41310466426765272,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.473,1975-03-12,true,1200505546.69,17:19:8.12 +1539417764,66718853370266608,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.473,2002-01-22,true,949928305.019,5:51:27.26 +1141554400,49760629751304256,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.474,1955-01-02,false,859077418.471,12:42:5.50 +1545516000,73847076151029968,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.474,1957-02-19,false,645829174.073,18:8:26.18 +600237027,6562593395657267,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.474,1964-07-13,false,661490680.733,6:16:47.14 +-1489402760,2082609175581634,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.474,1958-08-14,true,736790177.362,22:19:1.39 +1310234467,82819700021170128,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.474,1964-02-17,true,1282421285.05,5:34:27.17 +-645807548,32311305129853624,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.474,2012-10-14,false,508440114.365,3:20:43.52 +176048992,12998351345244774,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.474,1960-09-05,false,688819218.971,14:35:47.33 +-1666576224,83567278453682032,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.474,1998-06-12,false,371605454.354,11:16:58.41 +-1667419058,78355974898634688,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.475,2013-05-25,true,1071179444.11,11:15:3.12 +-941883359,73021992658218592,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.475,1958-02-15,false,96554096.7369,2:32:29.3 +420697994,49642284168281480,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.475,1961-11-19,true,547430618.204,7:6:43.37 +-1504961372,47884337435535440,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.475,1983-10-22,true,871627399.916,5:21:36.20 +1244919391,57714104407454800,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.475,2012-07-21,true,1337328701.53,16:48:47.16 +-288205840,35734894558574376,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.475,1998-01-17,true,813228116.148,1:41:53.57 +840466137,66167075176727432,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.475,2008-08-01,true,1354596354.97,13:55:4.52 +-1754157091,31443989133247028,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.476,1969-10-02,false,532123943.26,10:28:40.26 +-1924149439,90503273541768720,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.476,1978-06-08,true,854768723.047,5:15:20.42 +518809656,80194937726245200,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.476,1975-03-24,false,1111114971.0,8:4:1.5 +-755787398,84657030162344256,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.476,2004-02-11,false,1233313878.22,9:54:9.37 +1468109703,66432590924008672,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.476,1993-08-06,true,1239766201.63,22:3:32.28 +1186447130,23448822813304844,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.476,1960-03-18,true,889336547.685,10:5:10.37 +-86714889,28214649014592380,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.477,1961-08-06,false,1042870090.1,15:11:28.55 +-849791902,23148252516936480,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.477,1966-09-18,true,1335507770.52,13:11:1.42 +-107366380,35213361870785436,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.477,1973-05-08,true,333695122.308,6:18:57.35 +2063159029,60819400077270384,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.477,1995-04-02,false,1017754012.43,19:18:46.8 +1331753242,38013364019347536,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.477,1970-07-06,false,1270296154.49,11:37:8.16 +-522521915,34782519166158460,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.477,2002-06-19,true,1348304181.29,8:26:13.34 +596798601,71322712147499872,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.477,1994-02-12,true,165307638.94,18:41:1.54 +1540543209,68510398918422408,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.477,2011-06-09,false,316839149.088,22:54:27.17 +851551079,59235713669860608,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.478,1978-06-22,true,238853229.096,13:35:54.17 +-182172305,51183686467484056,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.478,1959-11-27,true,364973095.067,5:33:23.21 +1218675288,81995340515106080,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.478,1953-07-15,true,557872661.118,4:16:9.20 +-181565965,10266922713707510,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.478,2008-05-17,true,1428753735.11,13:41:45.7 +-478703172,65513021668871848,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.478,1982-05-16,false,677947362.113,19:11:1.15 +1848771360,26120940233295320,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.478,2006-05-06,true,204552372.576,20:46:35.26 +-368567255,10894317799171902,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.478,1974-04-03,true,459030250.93,11:9:4.26 +44026991,60599212361962640,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.479,1960-06-15,true,1109047569.92,3:15:27.16 +74904720,36112553359849920,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.479,2005-01-24,true,478969854.441,7:22:51.5 +-1406330119,90730164690652896,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.479,1988-07-14,false,1329666418.6,14:56:54.5 +1337566831,46492151540876064,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.479,1987-04-15,true,1250273609.93,18:29:14.51 +650563711,37573672414719200,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.479,1965-11-25,true,389420364.411,10:22:4.52 +605433469,23636020866605488,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.479,1994-11-07,false,406011203.656,19:34:43.47 +269243132,1222531551561021,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.479,1995-02-05,true,669170398.002,21:50:58.35 +54314374,47663194507372792,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.479,1977-02-09,true,860456885.64,22:18:54.30 +-1319881553,76967699092644496,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.480,1956-03-04,true,391254707.356,19:50:55.11 +-1313522231,81176532205019664,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.480,1983-05-15,false,1201698259.84,13:30:56.57 +116230944,5324063860383949,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.480,1953-06-08,true,1266134449.86,7:51:35.4 +-1257123056,77108777425198560,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.480,1977-05-09,false,635999416.637,14:9:24.55 +1854110223,50375605782817168,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.480,1993-07-14,true,1177025192.73,4:49:15.30 +-939612669,43810847118721760,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.480,1960-11-09,false,1334346078.1,17:46:14.34 +572138062,83307723648678672,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.481,1965-05-27,false,580385318.464,13:35:15.48 +135698402,43927969244272776,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.481,1962-07-23,true,429246739.131,21:11:6.18 +-2056228097,5479485130791158,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.481,1968-03-27,true,1112265390.64,12:50:30.2 +-465038632,5656202949525934,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.481,1959-08-12,true,828336944.118,17:50:6.7 +-258160672,57002441924517232,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.481,1973-07-12,false,833466458.554,11:5:34.55 +1053334442,42856988607037632,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.481,2002-03-12,false,1211775763.3,11:8:36.7 +-113224235,70086260492644960,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.481,1975-07-10,false,1058343453.16,16:33:44.54 +508044018,1464860224011438,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.482,1987-03-05,false,938819707.173,14:51:14.25 +-832267259,67340428871936336,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.482,1978-03-04,true,535915948.9,15:9:35.5 +-139731280,77106809278026528,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.482,1975-02-21,true,1253352575.27,20:2:45.53 +-45973261,54816819154267864,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.482,1963-05-24,true,1247582279.85,20:39:6.54 +-658290838,36517537660465944,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.482,1982-01-11,false,2820528.08114,22:57:32.34 +1594773638,80689629691663568,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.482,1972-09-11,false,554724263.876,15:36:37.6 +-394716805,5680420049449871,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.482,1988-02-01,true,901515326.483,16:49:11.46 +-1907692179,33390242714468384,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.483,2011-03-03,false,17812278.1556,5:38:40.33 +-1889424501,6723557667800668,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.483,1958-07-26,false,1253028700.64,11:29:50.14 +1590690841,77489790995808704,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.483,2006-04-08,false,1305799541.12,4:14:29.51 +-1870240889,14704374539802982,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.483,1974-09-10,true,521993512.591,16:19:13.45 +73987837,66837935556401768,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.483,1967-08-06,true,1087401301.02,19:46:4.47 +-2127059028,89567118073513600,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.484,1978-04-06,false,1246206101.4,9:46:54.17 +1148657809,91292458849479200,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.484,1977-05-24,true,1417644789.13,9:10:36.41 +-1373337097,64738379350208648,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.484,1986-08-18,false,1364559616.72,20:57:43.24 +1210414351,14455206475088722,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.484,1969-05-12,false,385087214.477,10:26:41.48 +971846465,61715239275577560,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.484,1950-08-19,true,102717733.483,22:41:51.17 +292211689,14071769977286584,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.484,1990-08-02,false,1398102215.29,10:17:47.35 +555499050,88283694162556576,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.485,2005-04-12,true,1033726554.69,7:11:41.7 +-58253397,81919936324887792,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.485,1994-09-20,true,18956605.7965,20:53:56.29 +-242077021,69719742204796672,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.485,1999-06-18,true,1009103196.81,19:24:20.15 +-1175800685,2988058520342006,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.485,2012-03-08,true,1047096880.86,21:8:31.56 +324423052,30802773539638744,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.485,1992-11-05,true,1232978845.35,3:50:53.9 +-1320007524,6388352676830444,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.485,2001-02-13,false,989261874.577,13:7:4.46 +-1988141705,6414529460935393,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.485,1983-05-24,true,725148959.243,7:38:31.14 +-806035925,22791581334052220,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.485,1951-06-16,true,786684780.348,16:21:24.43 +-1879395931,17582100092989348,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.486,1968-06-15,false,61366668.2611,15:47:13.30 +366501935,77284637958179312,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.486,1973-04-17,true,365747227.938,17:32:49.23 +902036745,7078050970455665,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.486,2009-04-05,true,679074119.795,19:21:51.52 +-1424400258,84340756746536672,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.486,1971-06-26,false,277537723.288,14:57:21.48 +-1469626444,88814648081187456,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.486,1980-01-13,true,306762964.46,4:8:44.21 +388161507,72888172961290848,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.486,1988-09-13,false,335758928.301,16:37:37.26 +-1366613672,25110501361328140,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.486,2013-11-10,true,138748170.714,5:8:21.21 +-985631261,55041662730284768,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.486,1978-06-21,false,518639495.648,19:1:42.18 +-1696012758,38978088766824968,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.487,1974-01-13,true,313589843.157,12:14:14.21 +1051882517,62519201358028072,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.487,1987-11-16,false,1038531737.34,19:17:21.12 +1676250608,38834253669145400,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.487,1958-09-04,true,429339585.677,11:20:21.45 +-630998460,16779941678219714,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.487,1986-01-23,false,1116411226.02,14:5:9.1 +2018620778,72839185730968144,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.487,1960-09-07,false,981566773.248,20:10:5.12 +-1933338502,16084537300094648,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.487,2004-06-18,true,693497233.107,15:44:15.34 +1992968462,26340223409085616,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.487,1996-09-12,true,639084025.635,10:15:10.15 +1003464315,19469450801116776,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.488,1962-04-21,true,604851952.284,11:5:58.18 +-1148363616,80702031860201840,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.488,2013-06-09,true,877294054.351,18:55:23.9 +1624290292,22495282384403712,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.488,1989-08-15,false,1179449893.09,21:43:38.37 +-1662705798,45610220359060088,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.488,1952-07-10,false,540805679.039,17:46:13.13 +-1714230449,57314018184786792,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.488,1986-02-24,false,739142280.829,9:6:13.12 +1426292185,60429453528142064,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.488,1983-06-18,true,1127008220.7,3:33:11.36 +852778487,43512262154722184,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.488,1988-07-07,false,670775889.822,7:49:52.10 +-76630556,16563762851976234,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.488,1980-01-03,true,1344680861.69,2:27:49.32 +34865876,9451202748491694,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.489,1974-02-04,true,228603844.649,1:49:49.48 +868528145,30607271853120176,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.489,1994-03-27,true,269788598.032,4:49:28.57 +-954082596,42597711722552416,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.489,1953-02-21,false,860685761.617,8:20:14.32 +-187299099,19738798881504072,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.489,1964-09-12,false,1319452009.5,1:15:22.3 +-2115186731,77541697663134288,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.489,2010-01-10,true,915547103.263,12:39:42.52 +-1784963354,14829048655957350,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.489,1965-03-09,false,1287157598.24,15:25:56.22 +366530793,66462575710693112,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.489,1987-10-20,false,102808380.217,4:25:3.38 +473809860,43642758800176136,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.490,1988-06-09,false,654675206.633,7:26:17.34 +988315895,64676044409686056,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.490,1955-05-14,false,1080616470.11,6:41:9.50 +1848610445,2278304379604162,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.490,1967-04-09,false,808995984.525,11:7:46.21 +-1786472480,3627083909087652,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.490,1962-11-25,true,633467472.916,8:3:34.37 +259097267,55819023129251936,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.490,1960-01-25,false,47625581.4373,9:22:26.23 +299827032,18212819238199020,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.490,2007-04-21,false,932847759.852,7:30:21.5 +1119472466,69858613074596392,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.490,1959-10-16,false,386126041.255,20:11:40.38 +-1154702877,90831567566602688,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.490,1997-11-23,false,658064752.409,22:47:56.38 +-66884552,16091279986200106,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.491,1992-05-25,false,1272888529.62,13:42:38.9 +-312645723,36705285660316016,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.491,2000-09-01,false,1371027383.85,17:38:44.58 +-2122573161,91000551966043328,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.491,1986-02-08,false,33395516.7285,9:9:25.35 +384689334,10500763402461646,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.491,1982-01-14,false,699291518.127,18:41:21.38 +1402486252,20607980930998508,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.491,1958-10-23,true,1268751678.33,18:45:13.33 +-125823614,12243349975992976,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.492,2010-05-18,true,50327208.2885,3:41:28.49 +468309697,10867007496014694,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.492,1970-01-06,true,994133494.417,21:54:20.40 +-1582359993,82517600844936576,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.492,1979-09-15,true,1406686464.04,6:9:14.3 +1947222333,91157936208998368,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.492,1983-02-15,true,1180331910.76,2:7:27.6 +1062997826,36207865286448528,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.492,1990-03-22,true,30118757.3061,1:26:52.53 +1819858329,63900164747798168,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.492,2002-10-15,true,242403157.606,15:39:50.31 +1222961877,23581749069029888,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.492,1960-11-19,false,784901087.535,1:51:18.48 +1418547793,61186168419224744,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.493,2005-07-19,true,948699462.445,19:45:58.42 +1584110484,38603948848581256,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.493,2012-11-10,true,1196747998.49,13:27:34.28 +595663701,34970043520656468,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.493,1971-07-25,true,689807780.681,8:1:31.6 +2070799130,35414554471428528,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.493,2010-07-13,true,576625109.977,13:17:52.7 +183099231,55990664036926656,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.493,1999-03-10,false,1265521430.28,8:41:16.43 +232915290,15775753920638168,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.493,1970-10-11,false,79490652.0363,3:6:7.1 +1722640278,24611820056970744,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.493,1980-06-22,false,126859494.093,13:52:43.57 +-1801251250,53102775540114344,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.493,1987-05-26,true,721429741.23,20:46:35.50 +1963794339,13676098860994744,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.494,1951-02-16,false,106021028.724,1:33:7.2 +-826992644,33625239853373388,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.494,1963-06-21,false,696412973.508,1:27:10.1 +1295353119,70392382223210040,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.494,1966-07-26,true,379873513.487,10:46:58.36 +-1756579904,23264059948269180,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.494,1970-04-13,false,1125393946.7,1:28:5.52 +138822661,25275797427830264,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.494,2013-11-01,true,236482027.691,12:48:27.33 +-1960664765,73924222168383712,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.494,2013-03-26,false,97616474.4321,12:28:50.27 +336292222,12818520564651868,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.494,1998-10-07,false,750901400.277,19:6:49.15 +-1836135859,9487077638695024,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.495,1961-05-15,false,522919204.306,7:14:10.11 +-1325515873,76545209926878496,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.495,2010-04-10,true,1275024360.54,11:22:30.41 +-1783097178,19575151785757152,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.495,1981-10-21,true,895890555.606,18:6:3.18 +1149799441,56810684124748072,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.495,1985-01-05,false,794770400.868,8:17:14.5 +1154590056,20859411470406964,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.495,1977-07-01,true,245735291.277,1:16:56.15 +1247517071,3151174222544937,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.495,1973-07-09,true,1354273227.65,17:16:30.40 +85647288,9606704067736034,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.495,1985-05-20,false,1291183953.28,16:39:49.10 +-676700605,25061423749364308,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.496,1967-04-18,true,360105096.035,4:51:58.6 +796905911,25896236238318860,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.496,1970-11-10,true,1419349188.67,1:6:30.5 +-1403689940,25188050076883240,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.496,1972-02-22,true,426306302.245,9:35:5.43 +-178821795,90000656348011584,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.496,1979-09-03,false,84243247.2366,13:19:16.35 +-34484057,74766057200278544,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.496,1966-06-16,false,643219208.72,14:14:30.33 +-170210984,34253934137118620,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.496,1992-05-04,true,96984312.1326,13:41:16.47 +437234225,12047879939972208,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.496,1993-06-24,true,1289125194.83,14:41:42.25 +-1001115902,40978631237014744,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.497,2004-05-15,false,1201140423.29,17:56:54.16 +821898773,60782952930987992,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.497,1957-06-21,false,1027048572.98,12:52:39.16 +-1386689838,10426916669701366,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.497,1962-10-07,false,1395038698.31,18:28:54.20 +404954532,88226518628934192,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.497,1968-03-03,true,1042475841.22,21:42:17.50 +-866212946,63290082412847800,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.497,1963-08-17,true,838983434.63,21:2:48.40 +1363728910,33220626506194424,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.497,1969-05-27,false,178845895.878,15:58:31.33 +-505802396,77972470503625088,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.497,1980-10-24,true,1407787622.22,13:58:2.30 +1378237973,51489778223377184,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.498,1984-08-03,false,1344187829.75,13:53:34.31 +-1394637313,76535503582058992,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.498,1961-02-14,false,518951721.961,16:47:14.49 +242373336,89881116116298368,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.498,1952-09-21,true,990935430.232,7:16:57.57 +1301319028,42886880612551632,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.498,1985-02-14,true,956858083.057,10:56:34.17 +-470676031,39661327674107864,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.498,1956-07-11,true,1283907143.58,15:19:31.12 +-1132420159,81500361081391136,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.498,1957-04-07,true,1399869643.06,13:33:6.8 +1920424300,38615143143988768,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.498,1966-10-25,false,496026583.649,22:47:9.20 +-1042669393,82105100000451696,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.499,1979-09-18,true,712599127.343,19:1:5.55 +-1160758453,91438807369745424,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.499,1956-04-12,true,764852285.262,4:53:36.3 +-212028468,85493127906742512,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.499,1954-09-03,true,205509865.136,17:23:20.55 +-1079820490,48296902118405712,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.499,1961-05-18,true,1092241223.74,18:52:53.4 +2134268283,53191816535271120,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.499,1953-11-22,true,516049731.587,6:50:45.55 +-1246782748,49473386318993264,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.499,2010-02-10,false,605934651.261,17:24:7.26 +1604765227,48036071565038072,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.499,2005-10-15,true,8544237.29422,10:21:27.28 +-2130870838,63895744770702344,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.499,1957-02-19,false,318477843.837,20:18:12.8 +-1627145918,31936012428842976,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.500,2010-01-21,false,1212784450.73,19:6:28.16 +616259939,63131731217662520,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.500,1984-09-10,true,295475438.963,18:22:2.8 +-1523149083,2819686522716784,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.500,1956-08-26,false,772368936.397,5:44:29.22 +719213768,85873648561138672,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.500,1968-09-19,true,137023772.047,8:24:4.53 +1252143804,16454830727360922,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.500,1950-05-23,false,334363163.503,13:6:2.42 +-1020924006,79493881349954000,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.500,1997-06-11,true,1172199501.23,4:40:57.52 +774014012,5482124395498609,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.500,1979-04-10,true,982324603.896,17:23:47.50 +-1623024416,69327555867693120,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.501,1976-07-18,true,469934111.471,20:32:26.42 +1477908375,69558403749253,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.501,1965-01-07,true,852562943.759,13:42:15.54 +417074726,2238029189574185,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.501,1978-07-25,false,83259255.1294,15:40:51.36 +788262081,21074306574036316,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.501,2012-03-24,true,440203815.278,7:58:23.33 +759315388,15077706140059300,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.501,2013-10-20,false,418700605.828,7:26:3.2 +-1264992913,81229136196766704,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.501,1997-01-14,false,725903412.032,10:8:20.37 +-1045711829,14585291782910690,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.502,1986-07-02,false,697855935.526,5:5:2.45 +783061955,74110068749952960,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.502,1954-11-26,false,373015396.53,1:37:48.40 +1757754371,1413824336630231,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.502,1961-09-07,true,1166359942.3,10:52:3.9 +-1322066431,37690295844715288,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.502,2005-04-16,true,300535868.972,12:4:23.55 +2014738591,18301489969206212,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.502,2007-08-15,true,1320628840.61,12:2:54.2 +1706341140,83838997557542832,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.502,2008-11-19,false,1343443915.45,16:43:47.22 +-1955418121,71411809982382904,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.502,1962-01-05,false,1139805973.46,9:48:20.36 +1831554144,52406202403755808,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.503,1961-06-21,false,576322355.631,14:40:34.16 +-123805124,85399029472411792,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.503,1975-06-10,true,368834491.428,16:28:14.24 +827912384,56693695459424104,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.503,2011-04-19,false,1063194862.17,3:2:56.5 +610289021,44291657856179704,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.503,1964-10-27,false,1384062191.0,5:42:48.3 +-426224071,36692024062276360,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.503,2012-10-03,false,374652440.13,11:49:2.19 +968671902,12658734694959002,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.503,1996-08-14,false,119561903.971,5:49:50.9 +-1967599901,522681699283947,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.503,1979-10-24,true,612591171.875,3:48:10.49 +725069972,1442835061232967,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.504,1955-01-04,false,611686494.025,14:37:26.33 +-921887484,15438390969303502,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.504,1987-01-02,true,857442137.945,6:20:26.13 +325790516,86652918018531808,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.504,1980-09-26,false,893117294.711,10:58:29.14 +-1601001826,85045938824150336,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.504,1963-01-19,true,796539332.875,14:33:21.16 +-920274179,85996245186884176,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.504,1983-07-14,false,823316823.833,6:35:48.21 +-1833617886,6256323730597735,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.504,2009-04-14,false,164611012.384,9:33:49.38 +1723848403,3100839033974733,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.504,2007-10-26,false,704757617.534,8:56:4.51 +-2048358404,77687260422798288,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.505,1959-11-02,false,208233161.497,7:20:20.32 +-324741358,29415515139229972,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.505,1984-08-17,false,490913567.968,9:36:19.30 +-1640471955,76684024310224400,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.505,2009-05-19,true,201709646.703,8:4:47.9 +-492057514,58161668900345888,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.505,1987-11-13,false,951878969.019,21:49:23.36 +550496098,28261112308118048,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.505,1968-01-13,false,315004206.044,17:52:31.18 +-1016328546,88429772800898016,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.505,1977-06-12,true,793825166.756,17:47:33.11 +2088551605,84579237883190400,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.505,1989-07-13,true,681094190.098,7:17:29.40 +30356365,36435869970776168,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.506,1991-10-01,true,215969091.436,10:26:12.9 +1658170065,39381075948299840,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.506,1970-04-24,false,979095426.509,21:45:19.26 +1718514931,14485859266008688,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.506,2006-10-11,true,1374363312.87,4:3:48.55 +-1865030916,72124049576222592,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.506,1973-10-13,true,848459585.461,16:35:2.20 +-157532561,91128332347284960,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.506,1996-08-22,false,109550920.866,1:58:36.45 +1591801444,26058503081437756,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.506,1984-11-04,false,912667556.697,15:54:4.12 +58211497,29334218436745944,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.506,2011-04-17,true,1407010860.68,10:47:23.54 +-1912760294,3757032871193753,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.507,1973-11-23,true,101233581.382,1:12:3.52 +-1565223098,30495276647944796,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.507,1980-06-20,false,354756415.636,3:50:56.2 +1848785,55027302227512064,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.507,1961-10-08,true,60032904.8406,20:11:51.57 +-1509654359,87728429223395744,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.507,1967-07-01,false,1072831037.5,22:35:39.18 +-1186769108,25887078381345280,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.507,1960-11-09,true,622101688.442,22:2:29.48 +-2053821545,52099522336359688,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.507,2010-04-14,true,257569665.046,13:7:30.19 +-2029516437,36012278178401128,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.507,2006-01-20,true,1302805696.65,20:29:19.50 +978460647,74387254264505616,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.508,1951-11-18,false,1173233181.58,7:20:7.48 +1182929771,4352428492380180,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.508,2008-09-21,true,1142048425.63,19:41:15.33 +247443892,89911947423976384,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.508,1993-06-11,false,558843096.714,8:48:22.49 +-1917900081,44824359060913928,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.508,2001-06-16,true,1377174500.47,10:46:46.11 +1976298386,30434741426475908,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.508,1996-08-12,false,1047553618.48,11:46:6.34 +-4886547,73632876147242160,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.508,1998-05-16,false,218088282.096,11:45:31.55 +-1261741766,30485615383599196,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.508,1987-04-18,true,802056364.638,12:18:55.38 +1885851654,2420537946384240,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.508,1991-06-25,true,71761265.2409,16:15:24.19 +415890564,57998233335582776,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.509,1992-03-04,false,1218670910.96,11:56:23.42 +-1482838567,39749380345664296,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.509,1969-09-04,false,825396834.023,16:11:2.35 +-1352436247,52042816823006456,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.509,1992-11-16,true,1163092145.14,17:15:53.26 +990044361,3334893013670799,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.509,1982-05-10,false,824681847.553,22:30:37.15 +-2084476167,6339310737046968,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.509,2005-10-20,false,1428407322.19,9:15:21.38 +-2019004370,86656426484190480,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.509,2013-01-05,true,171672085.978,6:2:6.9 +-1989728923,4610105677408942,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.509,1956-08-27,true,184299032.3,21:55:5.5 +-1254825786,508599595747502,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.510,1967-11-25,true,936122488.66,5:18:58.50 +-304778478,33557467899196344,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.510,2012-09-20,false,371545099.576,10:30:42.13 +1003909764,25992483069991804,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.510,1955-03-17,true,221551420.855,2:11:38.24 +216897061,79622021238482640,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.510,1967-01-08,true,409379046.445,22:55:37.36 +1725642916,8555133497744568,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.510,1986-06-22,true,568010260.993,2:18:11.58 +-174711067,6591569780738540,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.510,2003-08-06,false,25534686.3786,10:54:8.33 +1375691726,77568287752332160,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.510,2010-07-12,true,314559922.369,3:48:29.14 +316969364,86095887659656464,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.511,1981-03-17,true,517802206.398,19:41:18.46 +-1828914596,57659026371541352,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.511,1952-11-24,true,564257293.78,18:49:3.27 +-1063981152,71632925514122776,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.511,1998-01-18,true,899642274.538,6:44:50.8 +1546610652,80346497092152480,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.511,1997-09-07,true,327925452.729,19:41:29.47 +-1337505896,67528631644998904,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.511,1976-05-12,true,270828297.28,6:27:22.51 +1009771236,80923015143067552,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.511,1980-02-07,false,1148794116.36,7:50:46.20 +-769545979,27749421966244712,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.512,1967-02-25,true,115926626.521,7:9:53.37 +-1028152692,62046259403123384,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.512,1950-09-14,false,4843172.89884,9:37:50.18 +-1590463581,12390497840672154,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.512,1957-08-10,true,586302703.899,19:41:28.50 +1168142304,41704140018106552,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.512,2010-05-15,false,608114726.19,21:51:29.23 +987379782,57126962789610296,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.512,1954-10-13,true,401943154.998,9:7:5.9 +1379507004,76686137579758672,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.512,1986-11-20,true,331886247.549,12:54:15.21 +-1791642519,9398336477358070,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.512,1968-05-07,false,13443531.8905,16:10:40.40 +-542920599,8484013322711204,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.513,1973-01-17,true,217931804.212,9:48:18.58 +436212595,67463663801745088,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.513,1975-08-19,true,488401536.972,7:15:25.33 +860405289,18718802572653416,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.513,1995-11-05,true,1402409666.91,3:57:38.35 +-189427453,66500988023658272,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.513,2004-02-03,true,163647118.928,13:16:7.10 +-1685516081,27661335106052640,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.513,1952-03-11,true,179613294.766,3:34:16.41 +-809962881,11500629019143896,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.513,1977-05-01,false,156802507.903,11:28:21.7 +-1514247534,41707512839070816,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.513,1991-07-07,true,1370442463.9,4:55:53.25 +-280653079,12291908133668444,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.513,1992-02-12,false,625110838.369,14:4:23.15 +274281804,73852028754492416,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.514,1988-07-02,false,228563701.655,20:24:3.50 +-86717987,85844047679116800,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.514,1967-03-02,true,418723837.146,15:12:54.18 +-410991055,34226232251088864,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.514,1997-11-23,true,111583229.97,6:53:39.3 +-1802285737,59049802581283248,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.514,1985-05-17,true,855334094.7,12:3:29.39 +-1829752840,86309506044980896,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.514,1976-05-04,false,893219454.767,18:23:30.56 +-1860574520,44637268755350096,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.514,1999-02-04,false,1084818973.34,12:48:34.6 +-1851010386,67498675409015280,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.514,1992-10-21,false,1084406531.38,8:52:1.46 +1803834329,44000358019105544,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.514,2003-04-02,true,46570078.1193,9:51:41.53 +658005819,50490246739392544,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.515,1997-05-16,false,109043395.944,12:19:43.36 +-1266371785,20027989831772476,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.515,1990-09-16,true,803796764.618,19:22:47.41 +-996687372,9265302659370118,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.515,1967-08-10,false,562808168.967,15:14:50.22 +-1168977192,18170611455051500,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.515,1987-09-01,true,220687223.133,11:36:48.34 +1760958580,38495486456321320,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.515,2004-04-22,true,833011510.637,16:4:43.9 +206410382,49469823531831744,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.515,2006-03-09,true,1374743116.28,3:42:51.24 +2111092843,77653739160988048,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.515,1987-01-22,true,385101621.01,7:18:29.29 +-1862471462,64214252861024536,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.515,1989-09-22,false,439516190.012,12:56:20.57 +-1964649986,50286478722764440,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.516,1975-02-18,false,173790295.041,19:27:16.56 +-1792304174,32003582856079636,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.516,1960-11-03,true,809199244.106,17:5:23.36 +-392587576,79178783504820992,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.516,2001-08-25,true,984955952.185,14:30:26.3 +-1967367777,12158663996088,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.516,1984-02-20,true,326619066.727,6:7:50.20 +-912321766,21330643715470816,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.516,1958-11-16,true,368793348.1,11:3:55.50 +-3467401,30510174875045928,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.516,1961-10-20,false,274965681.335,11:20:32.19 +-975270688,46346381660346656,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.516,2006-05-27,true,159698419.026,6:55:41.45 +1478280234,11363143227502152,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.517,1974-07-19,true,474766881.509,7:6:16.48 +-1013323054,12040870069408112,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.517,1969-08-26,false,1286387777.49,18:43:55.50 +-873115511,7920203277115771,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.517,1951-07-11,true,1324210093.35,8:7:20.21 +1261063751,62918850048118776,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.517,1960-02-02,false,289269250.842,20:14:7.11 +-744181124,11678379639164714,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.517,1982-02-22,false,951347035.686,5:52:56.29 +-1990705735,45881550135245576,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.517,1959-06-16,false,1053093859.29,5:57:50.40 +905233028,86231939635176368,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.517,2013-02-03,false,1223019527.66,2:44:46.35 +856505289,65284938147219816,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.518,1950-09-03,false,204431422.648,7:27:28.2 +744281625,80870421934071824,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.518,1965-11-02,true,530346390.327,15:16:32.39 +-1549004674,76482720958438496,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.518,1966-08-02,true,324108772.329,2:28:9.48 +-798424275,40816073633951304,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.518,1971-11-25,true,905341438.059,3:3:18.35 +-1052637777,82418455394792176,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.518,1994-04-19,true,238210743.141,16:38:53.33 +-1175291614,46708758344766608,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.518,2012-09-25,true,311885505.403,11:18:29.58 +-1599891550,14611073674553026,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.518,1969-06-19,true,828001061.5,8:48:27.37 +-214929931,23411564603548896,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.519,1973-10-15,true,1184624366.09,8:44:50.48 +-1217273932,71809303115865848,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.519,2004-01-04,false,103132625.404,21:10:17.3 +1838077833,2105123984791163,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.519,1987-04-25,false,381261768.969,19:14:29.1 +1495163784,48762294267426912,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.519,1982-03-14,false,1188562709.88,15:26:12.48 +568223349,35423500732671552,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.519,2009-10-23,true,297183301.734,13:9:50.10 +485244177,85932273661704464,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.519,1983-10-05,false,186710828.708,20:42:27.35 +-706311630,22802334253587508,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.519,2011-02-09,false,1414424828.58,16:30:7.58 +652348176,62854275001198560,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.519,1997-08-20,false,645503901.91,1:3:54.23 +-1595225122,71183132843998056,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.520,1959-03-15,true,975065682.482,14:28:23.33 +575099243,24186382713963080,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.520,1982-04-08,true,1424486662.65,15:45:15.8 +-109429440,3468076588413071,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.520,2003-07-19,false,1301113106.84,19:22:57.52 +-1141139241,19144833572845140,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.520,1959-03-12,false,1272417594.99,14:54:30.10 +-1078818113,88873415496346608,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.520,2013-11-05,false,228135891.229,6:8:12.39 +-2058342686,32912246963414324,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.520,1962-01-24,true,1228077755.19,11:7:41.12 +1933364335,41875477207235568,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.520,1970-10-22,false,629988653.9,3:31:4.51 +812672892,29427754473152748,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.521,2010-10-11,false,43223243.1743,12:53:34.53 +-634773686,88597816174815952,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.521,1977-06-07,true,317017993.686,3:32:21.15 +-1543415128,26226225784511396,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.521,1952-10-19,false,561212237.672,5:43:11.22 +1051546990,23449076787947212,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.521,1995-01-27,false,165296316.567,14:20:49.53 +999206258,84802708932690448,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.521,1995-04-21,false,1132228093.21,18:13:8.18 +151584902,20576627985527008,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.521,1951-02-02,true,1244644241.92,4:50:5.36 +-2073805884,4150236288730235,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.521,1983-05-27,false,155534295.04,17:28:5.43 +1269223625,65738546466225880,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.522,2012-11-10,false,1369381023.42,12:3:12.55 +946242519,19757363544083632,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.522,1981-11-24,true,1352907096.03,3:34:47.46 +-172971976,83179150983688976,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.522,1981-10-15,false,1205959356.36,20:36:7.43 +66453918,44337733997290992,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.522,2012-07-03,true,926095798.812,11:21:5.32 +2128927868,26188724262591684,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.522,1997-03-14,true,301273037.173,18:8:52.23 +-1805764269,59018276161379832,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.522,1996-02-24,false,1004788361.53,3:28:8.32 +-166905000,75589746068446896,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.522,1961-02-09,false,622358227.531,9:48:8.52 +1990587552,13669728873946480,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.523,2005-04-02,true,664932212.782,2:57:3.16 +-1490121688,29408957503705088,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.523,1989-09-05,true,34667246.9793,13:48:33.33 +834435330,48710880085880160,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.523,2013-05-15,false,1020902512.89,12:13:8.7 +-1511375040,29533545641277696,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.523,1957-09-24,false,758610747.47,13:51:46.45 +2024853350,66932999791122504,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.523,1965-11-15,true,312814261.437,5:2:57.44 +9716742,18782835924536116,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.523,1963-06-08,true,886887875.502,12:27:17.24 +352836884,40642680184405016,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.523,1974-10-08,false,417281890.775,17:48:21.39 +949672837,1010661537638492,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.524,1958-07-07,true,548279203.351,8:2:19.22 +-1198396161,76635618153848448,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.524,1968-05-26,false,931798655.204,3:18:12.32 +1663408439,26585510497495244,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.524,1952-05-19,true,958301816.208,4:16:32.42 +-502233178,33070598133781628,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.524,1961-03-06,true,1293178053.72,7:12:58.27 +1627292717,45199295924981968,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.524,1962-07-17,true,343755529.286,2:22:41.9 +-118245581,65409959275065088,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.524,1954-06-03,false,1412959032.49,10:19:33.9 +1388690197,29756267819168616,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.524,1994-02-01,true,859282629.258,16:18:11.28 +-1216505791,59942514780227616,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.525,2013-07-07,false,488859501.32,15:17:19.56 +-1476990780,30983648649235396,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.525,1979-11-22,false,710426171.564,10:46:18.45 +-1314902493,4709650253750108,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.525,2012-10-25,true,7638745.08174,4:24:16.5 +256962296,61329008056722576,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.525,1955-01-25,true,888275301.405,10:52:27.52 +6785457,61282627883468808,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.525,1996-03-05,false,615860856.375,22:34:9.21 +-1402231357,1090727648829091,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.525,1996-08-09,true,416969411.585,11:52:50.45 +-2125128322,76924356012865952,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.526,1952-03-11,true,580654916.367,16:35:11.12 +1508012700,79925297493199184,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.526,1956-02-10,false,836576925.898,21:3:46.56 +1716051771,44701683974125264,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.526,1995-08-20,true,226032739.55,22:53:43.34 +293709396,79090755836205952,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.526,2009-11-24,true,1278541223.61,13:16:8.38 +191864269,36016565964081892,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.526,2006-11-20,false,1074975428.44,1:25:11.18 +822679710,1229983906322370,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.526,1968-11-19,false,523233618.142,1:7:22.13 +2003521384,33825741975538484,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.526,2007-06-20,true,744268743.639,10:32:40.33 +-1682208397,73575072649894784,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.526,1959-01-06,true,146349504.876,6:5:31.31 +1003861555,28472718819125824,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.527,1991-06-22,false,18653319.7852,16:53:44.31 +1334177191,3377425588653128,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.527,2003-10-22,false,754957889.501,1:31:26.34 +938645326,85464098960941872,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.527,1993-07-25,false,610771151.048,4:48:12.22 +1722548136,55782726904104112,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.527,1965-10-22,false,540642597.707,15:7:48.55 +582429712,1873163961050286,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.527,1990-09-03,false,708624416.374,15:26:19.4 +1368379469,4946895580182006,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.527,1997-05-03,true,386941528.093,13:33:36.16 +187542679,76133860491144976,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.527,1967-02-04,false,415797956.327,12:58:35.24 +2002621620,54474928464057440,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.528,1977-05-11,false,1121310413.12,6:17:21.19 +1851669867,13538257786173748,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.528,1960-03-17,false,670670786.026,19:41:31.21 +1522762426,64059143044345400,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.528,1983-05-27,false,1427676345.0,16:4:48.19 +-1918902530,27227516880100916,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.528,2006-01-26,false,824815599.626,3:55:5.3 +-1300121052,47874869827371480,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.528,2011-03-25,false,1431282242.11,19:48:9.32 +-169352615,33310567204645632,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.528,2000-08-20,true,1305372982.32,9:10:19.47 +-1203809184,52404413217621456,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.528,1993-11-06,false,585033521.711,16:10:33.34 +-697698241,61987444349457520,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.569,1974-11-06,true,1237695287.79,15:3:16.8 +-1685322187,39567482682107864,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.569,1972-10-19,false,1006288834.67,7:31:15.42 +-1862276551,45104503893220544,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.569,1996-06-06,true,128410588.029,22:57:30.21 +-2688144,11417891892430254,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.569,1956-01-17,false,1358753719.8,13:42:25.2 +-132294572,31639685351905180,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.570,1953-07-20,true,179153593.319,8:44:25.1 +-16435020,23272775788721776,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.570,1960-05-14,false,387896422.021,14:9:16.38 +421801521,48825373663964952,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.570,2006-04-07,false,727715502.557,9:9:24.2 +-1508908783,29376249332188284,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.570,1992-09-26,true,539543740.001,5:26:22.26 +902864693,23477178845142600,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.571,1959-07-18,true,1225104004.73,2:55:45.29 +1977331012,56928166046116616,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.571,1990-09-13,false,1262421492.67,8:35:37.4 +-1943439025,5716389188997417,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.571,2000-05-22,false,550687308.51,18:1:58.32 +-697396617,19399398686520044,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.572,1975-09-21,false,1015991995.27,4:8:28.41 +-1732358800,72928207858242176,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.572,1963-05-02,true,760310528.488,3:17:24.11 +31967726,88275936701520000,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.572,2013-04-09,false,1061494762.09,17:3:43.26 +1984198022,18230039902698948,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.572,1968-07-11,false,392419917.201,16:16:31.1 +-876833454,31558582730216512,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.573,1995-02-01,false,169959145.662,13:37:29.37 +1858770271,25084757028770704,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.573,1957-09-08,false,948096642.152,15:41:25.50 +-276565441,44019270422876488,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.573,1984-04-10,true,42688063.0594,21:35:26.12 +991442310,75124157109874864,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.573,1964-08-09,false,1301215446.17,22:35:41.10 +-920563988,58310541493644752,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.573,1991-08-19,false,376273115.949,4:51:44.16 +2050148184,58292922679468016,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.573,1957-03-26,false,1099295529.42,10:5:13.13 +-2052552663,43328898740913304,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.574,1995-11-23,true,1223403757.68,11:49:53.8 +1966922584,53833043917833904,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.574,2000-10-07,true,439538166.344,5:25:35.49 +1840349270,26050197754562744,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.574,1981-05-14,false,1237199718.85,13:30:39.57 +-23863745,58890960166039520,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.574,2013-04-05,false,1246463245.67,9:25:16.27 +-1336903425,71549831756659520,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.574,1951-11-13,true,72938307.4259,5:58:31.51 +-2116943263,33263427600821544,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.574,1993-05-19,false,1263244081.57,8:5:55.45 +2076671772,89758003704554176,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.574,1973-06-13,false,1037618546.29,10:41:55.37 +-840015802,36930655094533056,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.575,1951-07-01,true,965897280.13,8:6:18.45 +-605126904,34416021005920132,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.575,1982-06-01,false,234706747.588,21:44:31.29 +2047160535,74592021349605728,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.575,1986-02-02,false,615286833.49,1:29:29.18 +-1824965806,85811977020950688,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.575,2002-07-15,true,928074835.082,9:33:20.43 +-1400881521,35194191283668124,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.575,2000-07-25,false,1141201714.2,20:31:31.14 +455867434,51072421543611088,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.575,1966-09-03,false,1311855699.89,20:25:33.48 +-1033780967,4430582277694955,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.575,1999-02-27,false,445389017.813,16:5:58.20 +1911209982,37790455585774432,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.576,1998-08-03,false,1054713470.5,3:53:57.1 +1226486327,54782334862807440,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.576,1957-02-09,false,526499730.022,16:58:41.56 +953537708,80102892043736688,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.576,1980-09-03,true,1124225188.26,18:37:23.45 +1827779652,37956987988526848,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.576,1961-07-06,false,893587843.977,14:52:14.51 +-86687413,15882212795984558,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.576,1970-08-19,true,1188723922.05,4:43:5.40 +1219116884,79566101677806912,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.576,1953-10-12,true,435615910.701,11:33:34.4 +-1480343221,41926493772223960,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.577,1978-10-05,true,253672005.364,4:58:7.18 +1621264107,46224951663836608,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.577,1981-03-14,true,1064730074.37,5:22:13.11 +-794999752,8056906537817539,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.577,1970-08-10,true,1231032403.38,10:5:14.32 +1180993612,37485618235181064,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.577,1958-09-18,false,218898178.132,11:29:24.21 +-562153282,59375346151356368,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.577,1988-10-12,true,377494478.415,7:33:50.38 +1522476735,40393315685728752,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.577,2010-03-07,true,681689313.239,20:2:23.17 +1316507269,23073666941081068,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.577,1992-08-02,false,267391894.534,11:8:56.50 +-888748138,50131202749715896,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.578,1961-08-16,true,602911344.891,20:38:4.25 +-408354342,8824963417666396,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.578,1988-09-12,true,539562135.202,14:4:1.17 +-1074721025,6111020075764111,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.578,1996-10-06,false,844794585.595,1:28:23.22 +1096156075,91883669997866704,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.578,1978-09-22,false,1297427767.79,8:17:1.7 +-1045135699,84360540870845392,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.578,2006-02-09,false,307937881.405,15:44:57.44 +-1638227685,70928396652217112,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.578,1965-07-03,false,555446802.826,2:29:27.26 +-1515747809,49726442027506040,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.578,1999-03-16,false,222674535.168,17:3:14.16 +1675504374,57301247567282648,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.579,2012-03-19,true,699325489.609,9:44:23.40 +-188548598,14308299782533940,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.579,1958-09-11,false,127330498.9,20:37:43.38 +1726063119,13403368393098906,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.579,1968-03-19,true,1212734843.44,16:53:53.39 +-1643861186,42912366544231312,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.579,1964-01-23,false,135042117.34,7:3:31.58 +-173574018,38364282896161824,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.579,2013-09-15,false,114082084.639,13:12:36.38 +144636078,52016891688040552,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.579,2010-04-01,true,70295884.5828,22:32:46.32 +-2113536111,26028580543528132,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.579,2004-03-13,false,709977057.884,10:19:38.46 +874784282,24130073024317204,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.580,1971-06-07,true,1021720840.91,4:50:10.13 +305304493,26785483123043656,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.580,1951-03-09,true,52710267.3574,1:14:6.45 +1686634205,62608894420409784,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.580,1975-03-18,true,1144598120.81,21:43:44.33 +-1310825929,1698410758262118,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.580,1996-01-21,false,399978709.363,14:35:4.40 +465573394,21231809342158140,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.580,1992-10-05,false,941555748.132,6:21:43.18 +1405538109,26288338623997040,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.580,1969-02-12,false,28581298.2217,2:10:9.29 +654360121,31632662855702192,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.580,1963-08-09,false,176850775.782,17:43:19.16 +-411852000,5568987118900265,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.581,1979-05-07,true,144408385.721,17:35:24.53 +281622200,76497902565458496,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.581,2001-06-21,true,519594377.32,1:13:12.48 +2087568794,62147210478670656,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.581,2011-07-17,true,1361049542.84,12:33:36.39 +1760196570,14117147365581374,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.581,1972-02-10,false,917996519.654,7:44:12.30 +470343150,40476064901018360,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.581,1959-02-24,true,389903142.854,5:45:6.7 +1975930221,81091475452376128,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.581,1982-06-11,false,1192039011.21,5:4:47.25 +1630375062,17079623748352768,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.581,2006-06-03,false,1378731898.47,8:49:51.26 +-2030587469,4272790231113728,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.582,1996-03-07,true,91387074.0071,15:20:14.2 +-626005388,72260942451959424,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.582,1986-03-03,false,648768267.056,15:50:43.16 +-1207565435,89286157139733792,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.582,1970-04-22,false,740601302.751,20:35:32.32 +-1373387009,46840263254333288,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.582,1987-11-13,false,1336898100.24,6:58:43.38 +614079360,25164063187225324,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.582,2000-02-05,true,1430765024.96,9:54:10.34 +-553594544,91488498845308640,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.582,1959-02-06,true,109215623.837,3:49:43.34 +-813622635,27565259731055628,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.582,1991-08-23,true,492914084.409,2:55:49.18 +-1111610923,21016474674947524,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.583,1953-04-13,false,655333882.489,18:19:12.41 +-767971717,74381280376814400,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.583,2006-08-20,false,1140821231.91,18:58:19.52 +1871141061,36881918102508400,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.583,2011-01-10,false,174242167.313,13:19:21.47 +-702513518,71620858822999544,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.583,1977-08-21,false,693392636.881,19:7:40.21 +-778004355,30167734720395256,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.583,1980-09-09,true,784750380.657,3:8:48.17 +1028588358,78330506185925376,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.583,2002-04-02,false,118273220.575,22:2:36.20 +-2079596611,48770379505609464,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.583,2013-04-16,true,1302187659.45,7:46:37.20 +280035066,57256082068819928,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.584,1989-02-22,false,893078970.203,12:13:44.53 +2081025709,18579339006881332,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.584,2012-03-10,true,569911004.83,22:33:31.55 +-1917994778,11959555108623770,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.584,1965-04-04,false,1314597671.6,12:13:20.53 +765724322,92220800535642000,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.584,1951-11-08,true,534822044.867,10:51:21.22 +130357031,52661536956164576,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.584,2002-03-19,true,162615302.434,9:46:1.55 +-502256030,17399904780523306,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.584,1951-11-27,true,154581719.073,9:24:36.1 +1786938815,36312312934833088,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.584,2004-07-07,true,865352267.381,19:54:57.46 +88955258,1311269479154811,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.585,2010-04-02,true,845375375.532,4:45:53.53 +-1607778207,73401858386668640,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.585,1950-07-20,false,805242811.737,8:40:19.12 +357908989,62247929084178832,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.585,1950-10-08,false,1260701850.34,16:24:44.23 +1595393513,55451986217004760,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.585,1996-07-15,false,15607230.9026,2:35:13.10 +-160885717,58963819600649576,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.585,1988-07-06,true,494060602.116,4:30:40.52 +247000400,60733127391636776,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.585,1998-07-23,true,864289884.003,4:4:38.16 +-883427384,82761280540393760,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.585,1997-08-06,false,1251647761.41,1:51:12.14 +-1699787315,31573406763815692,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.585,1983-08-12,false,834917536.864,15:52:24.53 +1137662003,11408486506811392,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.586,1954-04-20,false,124330211.955,3:8:20.19 +-476886090,44807655432665112,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.586,1999-11-04,true,824776816.725,6:42:41.13 +-1887689868,78753802561739760,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.586,1958-10-23,false,687604844.774,5:53:23.16 +629865670,29319380164781140,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.586,1962-07-06,true,536499709.803,2:42:8.51 +2137254195,79164070689863584,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.586,1954-06-01,true,1199437616.89,6:49:41.23 +-441488829,12575596281715918,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.586,1952-02-24,true,937644055.912,10:50:56.50 +618945210,87455922665256096,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.586,1958-02-08,false,1309545083.4,2:45:49.53 +-1499007437,25444676133624064,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.587,1966-06-27,false,911350738.255,17:24:45.37 +-413070673,31017284544651980,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.587,1989-09-05,true,145768371.64,2:11:47.46 +509510343,65585793087162296,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.587,1955-03-16,false,987164625.361,11:48:24.32 +-1266100019,60748157869543560,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.587,1977-02-23,true,477573069.585,20:40:38.7 +-742675003,3631461613046528,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.587,2005-09-19,false,835137290.778,14:10:22.25 +-1832679735,51687025907688688,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.587,1966-05-09,false,227393289.428,11:8:41.10 +95773683,29824142472529552,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.587,1958-05-14,true,789094523.368,20:26:38.39 +-758953588,20869927814105600,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.588,2007-01-23,true,1379073791.77,10:43:10.51 +328313669,3640578613166694,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.588,1967-10-23,false,296344914.671,16:30:17.41 +-1821533534,30826602957352468,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.588,1968-07-27,false,1172790827.38,21:24:45.39 +831483950,29632624972776140,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.588,1963-03-07,false,1061491809.71,6:57:21.10 +5140076,66774131772977904,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.588,1996-05-22,true,1051675649.1,22:27:16.41 +1427761664,13325597948818074,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.588,2013-03-02,true,1089824444.84,15:39:17.50 +-1571308247,91255824142651424,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.588,2005-01-07,false,706167068.741,16:48:34.29 +-534113014,49738242552927200,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.588,1973-01-06,true,1051231355.66,4:38:33.3 +802357247,32962806545210492,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.589,1995-09-25,true,228892316.533,2:4:49.17 +-422059879,33564707888769976,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.589,2003-04-20,false,1249177400.79,4:53:33.22 +1795376009,89635365621778944,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.589,1957-05-17,false,486847408.817,21:9:47.4 +-674545510,63850252708447240,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.589,1961-04-27,false,1234580096.65,2:15:45.48 +1833397519,69088173166205296,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.589,1982-06-22,true,1055766151.4,6:44:55.44 +1146160231,68752312644620840,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.589,1992-02-19,true,215345407.947,21:39:58.12 +1237355334,74479613165074736,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.590,1958-11-04,false,624312098.503,7:7:29.35 +1540081820,9485480054401578,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.590,1979-09-20,false,829426217.616,17:41:57.1 +1695582327,84960412031002848,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.590,2006-10-27,true,548175472.111,2:11:3.14 +1767296434,55650578958657752,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.590,1985-11-25,false,1295560238.32,2:26:34.31 +713790635,61691704814216288,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.590,2008-11-14,false,173360615.688,16:1:10.21 +142346903,81148626525336000,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.590,2003-02-02,true,419275320.868,20:18:52.55 +992801513,24472972489969244,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.590,1956-01-10,true,154123369.282,3:21:57.52 +1147368292,52541901836313344,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.591,1969-08-17,true,779139466.004,14:52:42.18 +-1477448743,91609773915903072,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.591,1972-07-20,false,1013763102.08,16:30:26.52 +1811662180,42496457438220288,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.591,1973-04-25,false,962855861.148,16:1:16.41 +1458214674,19745408874450176,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.591,1954-07-16,true,446748505.399,11:40:47.14 +-1744191344,69818340328030744,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.591,2005-05-16,true,1296477636.63,8:42:1.12 +-1516628031,33008266514882356,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.591,2000-04-24,false,902950665.545,22:13:25.25 +-1765620862,71383099036584336,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.591,1980-11-03,true,226566319.013,5:23:11.7 +-519867385,5536983531012977,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.592,1981-01-13,false,1305486570.16,5:35:32.53 +70397293,78450066982313888,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.592,2012-01-04,false,793433825.879,9:8:12.51 +-1185760734,39983929205901192,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.592,1964-10-17,false,190021907.706,3:5:19.52 +1804512952,9466459811431372,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.592,1987-04-27,true,1257439182.33,8:43:45.50 +-1689621516,16499330374686352,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.592,1951-11-19,false,424207233.397,15:31:17.37 +-2026694903,88436477641909632,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.592,2004-07-02,false,320990972.765,9:25:3.34 +-2142299025,27431595206978756,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.592,1956-11-17,false,773751426.726,10:11:46.17 +-263762253,25545726795533036,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.593,1980-08-11,true,141004086.889,9:52:45.30 +-1823191692,72976184321913824,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.593,1990-11-01,true,973580051.426,6:21:7.43 +-1639981102,65362063563969632,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.593,1962-10-16,true,132057956.355,14:37:14.49 +614519822,91569085663195152,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.593,2005-09-25,false,1341074346.55,21:28:7.24 +1150178649,34397967638707888,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.593,1994-05-17,true,372379532.836,12:3:17.28 +224062513,81559755745265728,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.593,1969-09-23,false,481790478.352,9:17:40.3 +-1780689204,73687559630727792,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.593,1966-07-12,false,530339100.009,10:10:44.27 +93363947,84309594905582624,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.594,1952-05-03,false,923772504.605,9:26:20.23 +-2127089760,29032038862520024,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.594,1980-01-23,true,1329308936.92,20:6:42.10 +1181140283,51561492977131224,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.594,1969-05-26,false,1162455000.85,19:17:41.56 +-938673787,90346927496926976,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.594,1986-02-10,true,314369211.436,18:23:20.58 +2127643937,74409426101423920,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.594,1992-03-03,true,1296590523.5,2:34:9.21 +161882690,42791324049527488,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.594,1970-09-13,true,1289141750.4,4:58:21.21 +-395866585,33431041482818520,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.594,1993-03-23,false,1054852245.35,14:50:56.53 +88084564,58873752907766008,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.594,1957-04-10,false,78763471.6742,19:3:33.54 +-2133895400,18437570566345104,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.595,1986-08-26,false,1373850091.45,13:46:1.57 +-1145769178,16045522078365850,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.595,1971-07-21,true,875750076.132,8:24:53.18 +430520235,77305850991501216,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.595,1985-10-15,true,867515028.335,18:52:25.47 +-1643219097,10363105317913712,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.595,2007-11-22,true,747336670.515,5:20:39.49 +-1547945666,64903281881583576,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.595,1975-03-27,true,285916228.342,7:16:9.36 +86277214,36751142591048672,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.595,1971-04-07,true,1403939028.5,22:56:50.26 +-735013813,47525393882204920,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.595,2006-05-24,true,731218050.298,11:35:22.6 +98639415,72881302711306736,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.595,1974-04-22,true,1370434576.09,20:25:33.16 +271894378,28103480681767272,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.596,1970-07-21,true,354805048.636,22:17:35.27 +410666258,73833596159548912,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.596,1967-09-16,false,1377732002.6,1:1:26.42 +2122391449,64774291475373720,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.596,1974-09-01,true,448839678.128,13:52:23.50 +-1326960020,30899496794087324,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.596,2005-03-18,true,287695880.951,21:41:5.28 +316859684,89766536892579968,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.596,1997-05-04,true,1029771901.57,9:36:17.57 +-2079408761,44446090157652880,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.596,1990-05-02,true,1077847789.28,14:6:49.30 +-835045007,21237556227138816,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.596,1968-11-20,false,41993435.3258,9:17:39.29 +1578274639,76174005038268432,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.596,1973-09-09,true,599556304.603,21:27:40.51 +1895670897,74663299618849504,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.597,2012-02-26,true,955486497.26,21:7:31.14 +1284480309,48462970136954216,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.597,1966-07-03,true,371499455.529,11:49:31.19 +1654535323,19329032898575288,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.597,1964-01-01,true,1389670436.03,18:34:9.58 +-1981912797,43366189824219576,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.597,1968-05-15,false,109426961.541,7:39:52.19 +1308208576,58494307747535688,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.597,1956-09-01,true,1400181875.87,13:15:39.8 +1233031897,90518611649058224,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.597,1981-02-12,false,1112664.44408,14:13:21.10 +314872963,21004332004720816,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.597,1957-08-23,false,260692749.685,17:7:9.9 +-1217536027,74963687485152608,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.598,1957-03-19,false,1225033252.03,4:11:47.32 +1789433375,50419693194661552,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.598,1990-02-27,true,350902679.739,17:41:6.8 +546631709,82609751531530656,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.598,1988-01-04,false,201708484.225,2:45:53.24 +-675888682,89369486609088528,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.598,2003-10-14,true,905369705.48,17:19:40.46 +-140907212,78867126997495584,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.598,1971-02-17,true,1175543069.8,7:58:43.32 +1022313936,52407524286611152,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.598,1966-07-24,true,1143826985.86,3:3:11.14 +-243425638,36235566788748832,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.598,1950-11-22,true,426510998.467,10:40:44.7 +-1186936230,45110670039610104,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.598,1985-07-06,true,1026762111.89,22:31:40.58 +-853701849,16055932512244746,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.599,2004-05-09,false,1243220539.16,17:41:43.28 +330556539,45200659499292864,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.599,2003-01-16,false,1399582449.82,10:56:25.49 +-1513105768,88566080231094400,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.599,1954-10-26,false,5891355.2267,13:23:14.29 +1060599641,8197077942808412,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.599,2000-09-12,true,198307834.608,20:49:36.27 +-235693837,78800148906838064,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.599,1981-08-08,false,1246902872.61,17:32:27.57 +329480290,72974957283106320,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.599,2011-05-02,true,1424362568.51,15:58:25.1 +-1552414272,35361854677998244,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.599,1971-11-09,true,1146239600.53,21:38:33.17 +982054858,41837486454037384,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.599,1959-07-19,false,940824771.516,10:15:54.37 +-844699339,59222585816404672,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.600,1969-11-10,true,781982532.329,17:51:34.47 +1298713351,60956235530967064,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.600,1992-02-18,true,1273099405.23,3:20:2.18 +1873072022,28750235315626140,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.600,1950-09-18,true,801684516.542,12:13:3.7 +-1490433849,86792350097364352,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.600,1961-03-08,false,21098755.429,5:57:42.29 +-895320708,64503649449481688,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.600,1956-04-21,false,323967513.95,10:2:56.56 +536839833,44079555547129704,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.600,1979-07-23,false,601645527.262,6:23:5.10 +-74807215,64631638382881232,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.600,1957-05-06,true,37642830.4132,11:50:50.44 +-1889043602,19309210007333552,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.600,1957-08-11,true,479346960.002,18:36:51.41 +2036816321,30545192713026672,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.601,1987-02-13,false,1338369523.83,16:8:4.23 +1210921120,75711279452956160,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.601,1975-02-12,true,151637051.08,21:58:18.45 +-1228798288,35008155325078672,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.601,2010-02-03,true,1353996193.11,2:3:17.50 +-2130640830,88709746511309408,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.601,2008-02-03,true,54828619.6898,15:34:10.57 +-1524077633,65161713914458424,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.601,1995-03-03,false,810526058.441,9:21:58.13 +1522010929,54819358853947104,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.601,1988-03-08,true,1355239295.4,7:6:14.28 +-1531617258,13701290019778274,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.601,1994-01-05,true,1318890173.08,20:22:51.58 +-1221448029,52378999049939576,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.602,1961-11-17,false,560296251.433,2:32:8.52 +1571510474,37508606392017224,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.602,1950-02-11,true,797824010.227,4:18:39.39 +-1373768705,31978853639922976,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.602,1984-08-23,false,217388546.348,13:46:46.50 +-1839793405,59703343389690112,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.602,1996-02-03,false,473472963.974,3:33:12.19 +983784959,47804255627886832,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.602,2007-07-20,false,1316567953.31,1:36:38.13 +-2089262295,69628557002670376,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.602,2008-08-11,true,911621394.803,10:32:34.41 +-657551046,79419131086646128,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.602,1952-11-07,false,901795364.211,22:44:24.37 +108782680,72441998761539296,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.602,1964-07-27,false,919054005.754,7:4:21.44 +43625615,20073345219537828,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.603,1983-08-02,true,694378441.802,21:45:4.22 +1342421432,9744951592375746,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.603,1985-10-09,true,660090998.954,17:56:17.13 +646127505,50796009620254232,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.603,1981-08-02,true,293168685.145,21:57:16.22 +2127646615,71611493982606320,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.603,2007-03-13,true,183320352.416,11:15:20.58 +-1370143279,30331028236937708,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.603,1990-02-21,true,668801559.804,1:4:1.23 +-1086050056,77099195544620240,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.603,1955-11-23,false,894287331.441,9:45:45.39 +-643857391,22705819737866344,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.603,1999-08-02,false,1428888707.15,1:58:44.50 +-1237951023,92040474549470320,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.603,1990-11-15,true,519230316.993,6:30:52.54 +289818446,50445128763986512,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.603,1999-01-09,true,637988553.41,3:11:15.43 +2031109362,59249700906199096,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.604,1994-10-04,true,1072311483.74,7:58:7.19 +-676636030,62979023309935600,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.604,1984-09-18,true,1139951891.95,16:5:24.12 +853062363,64604612611373992,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.604,2013-04-25,true,825285113.581,19:19:30.4 +-2035811494,27345382387109612,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.604,1989-03-12,true,639721254.128,2:1:51.47 +-749376195,37360285055321392,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.604,2012-10-09,false,917188245.908,14:7:42.55 +-100156787,82795292493436784,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.604,1979-03-21,true,866801303.384,10:17:57.27 +-1441552944,64706675720317944,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.604,1957-01-27,true,1002746017.66,19:54:24.12 +-1054344272,45623003739421888,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.605,1994-02-10,true,1023241128.01,14:41:16.37 +-379137148,4367556957017119,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.605,1991-06-06,false,962463973.329,19:7:33.45 +1827406349,53983555559760752,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.605,1959-08-10,false,1264248174.09,2:8:29.53 +-1330261176,150458616136038,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.605,1952-08-15,false,407103875.059,1:28:36.47 +1909477756,74794628132001504,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.605,1979-06-09,false,665060834.283,19:24:44.18 +-171577928,85657635743157488,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.605,1969-03-13,true,637996166.229,6:8:49.44 +419497116,87987643169570432,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.605,1963-11-13,false,374839235.104,13:24:39.51 +-4417144,91776633216329120,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.605,2010-10-12,false,646810245.271,14:6:45.5 +842501468,81724539001229104,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.606,2009-06-21,true,1099205337.09,9:28:18.39 +-1327219947,42802977160738072,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.606,2009-08-13,false,777441162.92,4:52:26.43 +760451036,64667374152554480,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.606,2002-02-22,true,928025607.761,5:56:33.38 +-302394993,39635785131278008,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.606,1952-02-05,true,401448100.537,11:22:14.2 +1374503325,23327710736633036,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.606,1968-06-20,false,1365884135.99,8:48:46.6 +-58577599,12169956215685468,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.606,1956-10-16,true,702280737.04,1:54:55.35 +-692518884,8070041135878840,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.606,1980-03-10,true,715836232.455,6:13:26.23 +-1691366294,69345331378182912,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.606,1966-09-25,false,553288990.752,2:4:35.22 +414022171,91074655687399152,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.607,1981-05-10,false,1074924104.94,13:22:1.49 +-1957975547,87402198543192112,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.607,1990-09-06,false,1072371558.52,7:22:2.58 +-353503056,79159576929884880,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.607,1964-05-08,true,685359402.429,3:9:27.10 +-298030270,61748793482691720,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.607,1959-10-01,false,1126764178.48,16:58:2.14 +-1129998187,75895572257970928,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.607,1969-02-10,false,748612061.007,11:31:21.20 +-1335910949,4238956580368281,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.607,1999-03-10,false,929566146.33,20:42:28.54 +-1567457002,66670845943582208,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.607,1976-03-19,true,1165308513.5,21:19:55.10 +-1167370977,61093076505408616,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.608,1963-05-05,false,114144324.53,6:8:8.9 +922947897,58039026020039784,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.608,1952-02-27,true,756003919.394,12:35:17.40 +2053109438,50991015406843376,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.608,1977-06-27,false,117169050.163,19:22:57.30 +193087558,34179786830182268,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.608,1968-11-24,true,304536356.829,22:11:34.29 +1394926655,40875342825198632,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.608,1975-06-25,false,256823992.451,18:37:4.29 +1012457597,49017649285537448,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.608,1975-08-25,false,851730517.55,6:18:28.35 +1071672764,22106714673223036,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.608,1982-06-22,false,391927651.296,12:33:52.4 +-742787365,49568937383583152,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.608,1973-07-26,true,89834783.8445,10:29:7.27 +-625292965,36224163561536296,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.609,1962-05-05,false,257529876.92,21:3:8.8 +1141985753,69392939951894304,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.609,1973-06-24,true,834758731.128,3:42:57.50 +-57430255,30595299453490412,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.609,1955-03-26,false,1037777331.87,22:57:45.38 +-468234825,46605913833093312,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.609,2013-03-07,true,1100238708.2,18:27:20.25 +392344320,72396647740049216,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.609,1978-10-21,true,150493438.991,22:13:41.29 +-1661849429,56376851612770016,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.609,1990-06-04,false,1032664540.34,13:56:3.33 +-1211667615,2742309686403246,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.610,1986-06-22,true,1088761288.89,20:17:24.54 +626773642,73405178789332976,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.610,1974-03-27,false,220225402.707,14:35:40.18 +618010233,58124369154801296,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.610,1966-08-23,false,1099265330.24,2:47:12.54 +-955062287,28800050986755524,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.610,1985-09-20,true,1421268459.32,12:8:32.44 +1932858582,54738594024593736,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.610,1989-03-27,false,1237424222.71,8:43:37.46 +-11017508,84288808320564400,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.610,1987-03-05,false,709252467.727,7:21:25.2 +1134739260,75781649994297616,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.610,1978-05-13,false,1141861308.62,21:41:32.19 +-483370863,22812852263999036,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.610,1976-10-24,false,282334339.649,15:11:33.36 +-1826562582,42094109555725168,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.611,2007-08-01,true,543085205.645,10:21:43.45 +-861083973,48021624392797648,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.611,1981-06-14,false,1309561958.08,11:16:45.57 +1671045992,51495780947458776,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.611,1972-07-26,false,338002821.534,4:24:19.6 +824867266,3979379778427197,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.611,1989-02-24,true,576730921.065,19:53:52.8 +-30899241,90777463292199920,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.611,2013-06-09,false,654639360.839,4:57:50.10 +617567064,39783146110824704,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.611,1956-03-10,true,1253909170.82,13:40:34.35 +665825306,90153473138787040,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.611,2000-07-11,true,503686355.621,19:10:6.20 +-2117088487,88503419017032496,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.612,1951-03-11,true,787346910.05,4:28:27.2 +1370138228,1050398119176366,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.612,1979-04-07,false,231451389.016,13:17:34.6 +1797856021,20251775239034336,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.612,1997-01-12,false,1189101605.69,18:44:47.54 +1768946140,87544554398897472,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.612,1966-01-07,false,692382448.36,21:57:54.42 +-10423975,2408143649166479,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.612,2007-06-21,false,668840996.622,21:51:40.54 +965761401,64803054685329952,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.612,1954-02-07,true,94138640.7031,16:47:13.54 +-716702896,7457271515915541,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.612,1979-04-18,false,382139128.899,7:32:37.1 +584960473,72028460538631160,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.613,1986-05-27,false,683393560.224,10:3:17.16 +1730055927,68740423255380480,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.613,1965-07-13,true,308793941.506,19:10:42.8 +905293988,9787294979618858,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.613,1965-03-05,true,978908902.102,20:50:35.2 +523476441,59263886113672256,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.613,1968-05-13,false,969070285.947,10:1:7.6 +-1358961473,38168226611429824,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.613,1962-02-07,false,601457829.243,7:34:45.41 +2074831524,35002436529263884,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.613,2000-06-06,false,7699019.87431,10:58:20.52 +2144548638,85858347265304176,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.613,1998-07-01,false,1457866.65247,16:53:1.41 +1639858184,48758158655435360,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.614,2009-01-25,false,600268226.431,1:6:39.21 +-1793997740,36967429050238216,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.614,1961-06-26,true,898446195.437,14:43:42.17 +2137649717,20849737551601828,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.614,1987-06-04,true,834224878.44,9:53:22.54 +1368832768,21556066049595116,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.614,1998-11-11,false,296427221.68,19:13:55.20 +887880648,26352200072586896,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.614,1978-08-27,true,955928278.371,21:50:15.9 +820873336,89104136375280688,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.614,2001-03-24,false,566495217.21,15:29:32.34 +-2078221213,25395646689347624,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.614,1951-06-08,false,859586804.514,7:44:8.44 +553532419,85942981270779680,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.615,1966-07-09,false,1387826665.7,15:57:23.7 +996265045,21384913028446804,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.615,1982-08-17,true,661865741.637,5:24:22.3 +242914599,53466717343058552,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.615,1967-01-18,true,1191601873.99,14:23:22.52 +425976398,4268254355699558,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.615,1951-09-11,true,433224234.718,3:9:41.4 +-470178326,63054669885347792,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.615,1990-11-07,true,856652021.912,3:11:8.40 +1961647977,92150087870812832,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.615,1974-09-21,false,459385384.053,21:27:17.57 +97487185,88266175769152704,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.615,1962-10-19,true,527597262.803,15:25:54.5 +-1368186355,34606156294160168,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.616,1962-11-10,false,653931291.442,10:4:58.57 +823341059,35313604282933996,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.616,2004-01-04,false,1349260098.81,8:1:12.8 +-1856606290,65270577074266760,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.616,1987-02-03,false,1173175503.3,11:46:15.55 +1555038440,50425285578997256,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.616,1965-08-20,true,1176338143.99,10:19:35.10 +-1717260752,15861890197931848,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.616,1954-11-23,false,303344540.056,14:3:21.25 +-280373842,70210318911802648,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.616,1968-01-11,false,1036191074.73,9:17:50.4 +-1126228128,65250942710312256,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.616,1963-05-18,false,444724717.369,22:56:33.54 +1534083160,8619405625569628,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.616,1970-05-05,false,131521668.193,9:24:34.7 +-1601629712,55596529567510864,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.617,1999-07-27,true,671755871.843,20:33:46.24 +221789340,82641807595813456,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.617,1952-09-23,false,985637420.431,3:52:43.5 +40663848,90915427171570496,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.617,1988-06-22,true,149957404.03,5:17:23.29 +-1868469277,4672056454174577,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.617,1974-02-12,true,1400296021.88,11:36:42.42 +1235592010,36806654413609760,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.617,1970-01-09,false,874669490.485,22:3:6.45 +-2063768792,35738127938986784,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.617,2003-08-23,true,285514425.633,15:5:19.47 +300410844,34633011252689604,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.617,2007-04-04,true,637452048.753,13:30:26.42 +326868921,89604479675912944,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.618,1968-08-22,true,968021264.925,10:42:13.5 +1469528600,13240509395295980,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.618,1996-06-18,false,1002565214.29,15:13:51.2 +-832495467,1035422532790753,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.618,1984-02-18,true,902458262.673,5:15:51.40 +-277973394,45589079979153800,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.618,1966-01-11,true,883386170.9,15:32:6.33 +-187707953,49573759255597384,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.618,1995-02-09,false,141213358.666,4:29:11.28 +2087982763,85057234627910112,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.618,2011-09-08,false,751262101.926,18:15:27.9 +1432141119,19372112480591412,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.618,2003-06-22,false,592766419.4,17:5:30.56 +1233147099,23386069844948124,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.619,2008-04-08,true,67991049.6694,1:48:2.36 +1106427073,47771752695532760,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.619,1970-08-10,false,1053556435.62,1:20:18.11 +-1072327218,38170860821345720,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.619,1980-08-11,true,562155797.494,16:19:20.25 +-417122450,29000987323939584,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.619,1998-10-25,true,1358636323.42,18:10:10.53 +-1934598284,63572302189104112,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.619,1963-06-15,false,812738386.048,7:37:33.43 +1632676818,65553967606177488,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.619,2009-01-18,true,876512374.676,1:22:25.26 +402969837,42298187862956096,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.619,1981-04-25,false,118666637.886,16:12:3.32 +279067779,68861672947912216,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.620,2005-02-21,false,598059808.312,13:27:14.21 +88969768,77768111736041632,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.620,1974-11-15,false,1262159686.22,17:41:32.36 +-1771966878,78714218168299056,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.620,1973-10-05,true,937820767.581,18:45:22.34 +-932913323,34180432902463600,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.620,1961-11-16,false,404446627.581,13:25:21.35 +-1893170266,85954017908618672,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.620,1977-03-25,true,707141306.817,10:25:48.52 +-1724518700,51861352420249312,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.620,1989-09-14,true,150878803.557,4:53:17.54 +1907201734,84426384122672912,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.620,1954-03-07,false,282015097.101,6:19:22.51 +-823907814,30930858381006512,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.620,1984-11-26,true,358435833.73,8:27:38.17 +-1055223119,37912430037040896,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.621,1977-08-09,false,748940228.515,10:1:19.47 +-427142708,89805815226797808,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.621,2004-01-10,false,129322606.076,20:51:14.24 +1323781032,55864579111172752,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.621,1995-02-12,true,1070965668.37,3:55:11.38 +-409056180,42628056380324504,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.621,1977-07-26,false,561272396.8,20:36:46.21 +-1269765868,17253341567885098,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.621,1988-01-17,false,376266810.09,19:35:57.36 +-904179114,69372564378161728,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.621,1965-03-12,false,509975746.538,7:42:2.39 +45145760,91350663300082400,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.621,2004-10-13,false,723617256.866,7:13:32.50 +1420224521,18314513328998740,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.622,1996-08-14,true,1055751123.4,2:10:2.29 +84331514,20570181557682084,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.622,2009-10-21,false,1068703764.76,20:53:1.29 +1374630108,87905501290143616,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.622,1980-01-23,true,953301196.261,18:5:11.34 +-1776149710,38197730141631672,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.622,1961-07-26,false,1040405134.28,1:13:3.49 +1562587431,35336829580891268,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.622,1989-06-20,true,844942106.864,15:30:39.6 +-290070162,82997205425362528,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.622,1986-02-11,false,1327973570.76,16:39:21.45 +-1643820264,36078808525933320,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.622,1988-09-23,true,875701062.656,13:34:19.50 +-720168266,2996106890395935,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.623,2005-07-07,true,1357237924.1,19:30:12.30 +1266956148,78998290076687104,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.623,1990-01-16,true,615653513.438,13:39:33.13 +545280863,5884934272382710,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.623,1975-02-04,true,957047297.148,8:43:55.58 +2004852855,47617212090914056,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.623,1987-02-03,true,121867687.254,15:9:7.37 +-516697545,57061346114871400,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.623,1966-10-15,false,161449415.62,22:57:28.33 +1472077465,28168088734076992,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.623,1986-02-16,false,249582009.001,17:5:14.42 +-1257787749,24191146406129460,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.623,1969-11-22,false,1283371666.02,7:57:17.46 +213291549,25065165496759776,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.624,2001-10-11,false,800811052.764,1:12:19.39 +-2116970306,40532265346850960,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.624,1961-01-08,true,650938221.561,17:29:5.40 +-1180693870,81019557554039072,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.624,1964-06-02,true,1025807997.1,4:39:17.40 +-1401460139,10508627390859940,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.624,1957-09-22,true,781207216.272,21:45:52.41 +-59070121,89866879476392416,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.624,1981-08-05,false,863071210.884,18:13:53.39 +3894658,64499600562909536,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.624,1971-05-21,true,1266090541.93,22:43:23.34 +-1327433698,5520263535178465,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.624,1972-01-19,true,213337980.04,19:25:54.45 +529199897,13196154935965348,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.625,2007-02-20,true,561114864.192,8:32:50.53 +-1101196210,18321227123250044,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.625,1971-05-12,false,855010329.621,11:43:19.38 +-312467374,87912706020390416,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.625,2008-10-11,true,1067262146.53,2:13:26.21 +-1334462082,39863658769374264,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.625,2010-03-19,true,604857738.916,18:22:38.25 +-320373241,77577273197954352,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.625,1978-05-26,false,1057804264.29,8:3:42.2 +-1218596770,81948014271590624,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.625,1962-03-15,false,856613196.341,1:39:37.39 +1101339929,50783388127253168,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.625,1985-03-13,false,476341938.395,13:57:35.40 +-1635577995,89666595803519344,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.625,1995-10-03,true,1174571983.24,22:30:54.51 +1750026086,84474568496625424,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.626,1996-09-24,false,811167310.517,18:52:44.45 +730123939,58244761121837088,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.626,2012-11-04,false,691579023.627,20:23:42.16 +2112131896,47457597287215208,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.626,1964-03-09,true,81371433.9971,5:7:47.38 +-76394651,58182237716925888,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.626,1963-01-16,false,600577139.907,14:18:48.11 +-663183666,20938445542802760,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.626,1953-11-03,false,459435162.93,19:46:2.31 +1729630994,5929681109981635,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.626,1957-03-27,true,113623690.553,8:7:12.12 +-2038157555,80701828533700656,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.626,1994-06-23,true,246646823.539,22:3:17.18 +1852624447,49880080557016560,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.627,1955-09-25,false,700974136.458,15:31:38.49 +-1575041074,25426735300817276,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.627,1971-09-10,true,66378842.4602,7:37:18.42 +1232243224,2361079635462052,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.627,1996-04-25,true,719084116.972,6:18:54.8 +12167834,33113811504524032,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.627,1976-03-06,false,1154765398.27,10:17:43.36 +-149629281,9853371782982830,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.627,1967-05-07,false,672365720.958,20:29:8.32 +434126271,15552180161579736,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.627,1957-04-17,false,395232197.647,17:50:44.24 +-759227513,4280191280986839,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.627,1969-11-19,false,1350025272.64,15:45:45.51 +30111929,21144446995362160,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.628,1965-03-15,true,258901214.914,3:32:34.16 +-530497863,74793933597252496,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.628,2001-06-19,false,1369375053.26,3:20:50.18 +1898561021,88284517651314224,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.628,1978-04-13,false,885931449.39,21:11:12.5 +469675711,70397025183016368,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.628,1981-08-05,true,290445689.748,2:24:1.45 +2019991635,85947663978170768,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.628,1950-04-13,true,1247028645.63,20:37:36.45 +994903590,24482067711483924,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.628,1964-04-01,false,1152989299.05,15:50:3.13 +1642435323,45375737515807776,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.628,2000-03-14,true,824327391.454,11:24:34.4 +300198158,10657283023964702,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.629,1976-03-02,false,1031925855.48,6:2:45.40 +-1845807556,9276951422133780,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.629,1990-05-04,true,1199303698.27,11:6:26.35 +-888557652,44350753453651736,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.629,1959-02-21,true,1384571719.98,11:26:18.29 +-1444789337,86476036910522224,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.629,1950-11-25,false,679654171.937,4:32:51.8 +-657462502,25043533105340476,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.629,1985-07-16,true,388742358.004,5:31:57.53 +1673015766,10961247572471686,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.629,1952-06-13,true,564871416.865,22:49:32.33 +1455634870,66796227486273864,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.630,2006-10-09,false,898463511.054,5:36:35.52 +1118490659,79852953797473136,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.630,1980-02-05,false,1248096894.53,10:7:12.22 +-1732786716,63459789158575728,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.630,1999-07-05,true,1053831794.54,8:46:39.13 +-776844071,78704589964898096,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.630,1986-09-14,false,721378403.982,11:55:12.7 +-2145625965,21326957658108948,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.630,1950-08-01,true,228421903.227,11:56:43.35 +-546018851,58393957145679432,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.630,2008-06-04,true,560132452.996,4:9:31.7 +2037230744,67357830731613376,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.630,1964-11-03,false,1074012070.7,4:55:26.36 +-1585750051,21991999952001876,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.631,1987-03-25,true,783573688.601,22:45:55.43 +1685590952,4274677719828664,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.631,1955-11-24,true,717108428.718,15:7:50.24 +1800855709,66896639937572240,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.631,1969-11-25,false,1209552223.28,3:20:49.45 +1799449203,87628328383322480,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.631,1957-03-02,false,513982321.533,5:54:5.22 +-1331875915,74497173039447248,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.631,1960-05-06,true,1090061386.57,13:54:34.42 +-113202151,30832223301552560,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.631,1985-08-26,false,231686147.483,7:5:53.5 +-103840711,3702293806998292,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.632,1972-06-20,true,789497521.155,5:5:1.36 +-836495191,83950055139288336,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.632,1970-09-14,false,70656060.6607,1:49:47.29 +929387250,49882652838298440,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.632,1970-11-07,false,1160997229.17,6:58:34.49 +-2025702071,56120936617661440,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.632,1970-01-07,true,1077799690.17,4:46:23.27 +-1456535922,33232650907817440,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.632,1978-07-07,false,881142510.657,14:55:2.8 +-1307140174,355371873422991,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.632,1954-02-20,true,1110899999.65,4:55:25.2 +790678379,71250089937726816,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.633,1981-03-25,true,501683163.975,11:9:5.7 +1648561386,75560812529837616,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.633,1952-05-11,true,35323915.517,16:12:40.36 +-1031239672,51862993540715200,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.633,2001-01-02,true,1040140359.6,5:4:1.20 +1773166451,72022159880772104,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.633,2001-04-21,true,183268757.861,18:34:6.48 +-1554193761,42017274301354536,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.633,1969-10-24,false,214648008.414,9:33:53.26 +1204939855,10307876006292686,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.633,2007-05-14,false,1151364090.8,17:43:1.15 +-266266397,34134782600421592,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.633,1984-07-03,true,79008444.5353,5:56:44.17 +-830920262,18851326664888916,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.633,1974-02-25,false,538861031.709,2:9:49.24 +420047861,23853786386899844,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.634,1954-04-19,true,244211010.985,21:34:56.53 +-61167418,77554890845258272,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.634,1951-10-18,false,1236718112.55,16:43:20.2 +-1827077591,31244454071653284,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.634,1956-05-04,false,836466153.451,3:34:54.23 +-2113938811,61803830468078888,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.634,2003-09-03,false,973159577.862,22:24:39.43 +-1115541695,6957187287158160,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.634,1967-01-27,false,1377202394.24,9:33:25.53 +-1012157544,81064861415589568,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.634,2009-05-27,false,1076892884.65,22:14:44.13 +-2094290227,35912683587321252,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.634,1970-10-06,true,846541391.585,1:22:51.41 +2001111898,89247737138361680,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.635,1969-09-24,true,841142496.908,9:43:41.24 +-356096889,27568143583403512,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.635,1963-06-04,false,1090881379.12,12:55:33.15 +1707046839,84914428954699168,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.635,1997-02-08,false,866366743.838,22:2:31.36 +702737500,42469699559787376,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.635,1995-05-11,true,834750091.084,21:12:34.25 +-1199822090,8180489136218696,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.635,1972-01-16,false,799819377.233,16:36:16.20 +1242306503,56543773411445232,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.635,1953-07-13,true,228555275.01,4:38:24.18 +1777740953,60848546988105984,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.635,1995-11-18,true,1218304192.71,4:49:43.14 +-245872805,4695072798000374,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.635,2009-06-24,false,1330240135.23,10:13:18.9 +828919728,74721207900483920,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.636,1957-08-06,true,423754090.66,9:49:4.51 +-887830865,5939167356461374,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.636,1958-06-05,true,128410083.245,3:50:8.14 +-739281396,35024981219021700,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.636,2003-01-04,true,241880613.848,21:48:21.9 +-1588562567,38325531298427096,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.636,1959-04-08,false,315362076.097,6:46:4.15 +1917652601,43903258432225776,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.636,2013-10-07,false,123107458.246,11:49:31.28 +1556489001,42550426441032368,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.636,1950-05-13,true,1194399944.7,11:29:48.13 +545940923,57371352580769360,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.636,2001-05-26,false,1396060150.47,18:13:3.14 +123444729,9866841192342374,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.636,2004-02-18,true,945666927.356,22:40:41.7 +2090071919,8255829490058138,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.637,2012-03-23,true,1204235346.74,12:41:7.58 +-1421348612,45692274743601800,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.637,1958-07-24,true,1137188943.94,2:3:15.32 +-517078707,54742761653707664,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.637,2009-04-09,true,955922501.404,14:4:27.43 +-1803850404,41750498951515016,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.637,1961-01-22,false,1322230032.88,1:29:27.26 +1800996747,13886819984454360,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.637,1977-11-01,true,1346428154.01,15:41:32.5 +-1466030903,70019314381089608,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.637,1997-06-24,false,1413752973.93,21:31:51.6 +1391955551,66531033184049552,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.637,1976-02-04,true,1062381816.47,5:32:49.9 +-197661475,63561613838207472,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.637,2010-05-01,false,857528185.012,21:39:21.4 +2007908011,88213354062161952,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.638,1976-08-18,false,266213153.661,6:42:54.52 +1661804901,79165272317604256,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.638,1996-04-18,false,1320860522.7,18:9:26.32 +-1522599752,59354140374624216,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.638,1975-06-27,false,1019294569.68,12:56:23.24 +-789344063,1272712003840358,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.638,2005-03-06,false,931264671.574,16:25:31.44 +1558453260,71879487630806864,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.638,1954-01-01,true,1199805348.6,2:55:38.53 +-1221097952,25141970783110460,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.638,1956-01-26,true,1191882419.97,13:30:5.22 +-2053575113,16403982885402430,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.638,2002-07-26,false,1011948566.96,2:11:42.33 +-1333471444,24658476464254300,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.639,1982-06-27,true,421467885.632,12:12:22.38 +1345719979,22821955578696448,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.639,1978-01-06,true,374380190.338,6:17:51.22 +-542078750,27155369022441952,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.639,1963-02-06,false,183826537.569,15:54:44.3 +342887643,21347199579155128,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.639,1999-09-01,true,564617896.614,11:16:8.21 +-1313156894,45796622109502856,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.639,2008-11-16,false,42410135.2848,10:1:7.56 +-1726661669,61077897560306368,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.639,1964-02-05,false,227628322.522,12:11:44.58 +-1850462125,12008074927904124,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.639,1995-11-10,false,1144947123.88,18:47:9.53 +874297103,38354138454597144,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.639,1985-10-18,true,37833513.8038,14:49:23.32 +383233800,28802598028488980,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.640,1998-07-18,false,217568012.407,7:52:37.43 +732411287,35502557355796244,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.640,1978-01-14,false,440911627.287,10:24:33.28 +399883486,62603565152294080,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.640,2002-09-20,true,44050981.5838,21:36:25.57 +1706919701,9155847303029822,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.640,1952-11-17,true,933608191.333,17:17:5.11 +-1201398911,55009432671600600,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.640,2003-11-19,false,292137277.292,22:53:30.21 +-1733721895,75698675105184496,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.640,1955-05-02,true,720126366.508,10:44:49.20 +-299647608,8502748217009480,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.640,1966-02-27,false,839677799.613,12:38:8.50 +1379857411,69536853196281592,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.641,1984-03-15,false,1060883679.18,13:30:37.36 +-1067983942,32906526838679964,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.641,2013-03-20,true,275782610.122,4:5:38.45 +-1498680627,74237784694936480,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.641,1999-09-05,false,367366595.369,9:42:48.6 +-805315831,44214481411365728,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.641,1975-10-11,true,1112215203.47,22:2:33.37 +-1054149743,57912790718735544,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.641,2007-06-08,false,541525617.02,3:29:39.56 +-673237116,32630744916634380,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.641,1973-07-05,true,1251840140.62,8:49:22.41 +-433099198,91139683187446624,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.641,2010-09-04,true,1222820023.7,18:56:34.18 +1477715066,90702462928444192,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.642,1960-01-14,false,146963923.462,6:27:52.51 +-1401594301,45690698939313296,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.642,1994-09-25,false,1320118558.14,18:30:44.4 +1360221610,314145839661598,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.642,2006-01-07,false,848582981.068,7:55:58.46 +1974770060,24561485229437144,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.642,1986-02-12,true,1242230801.46,15:56:46.10 +347916314,34493988282510768,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.642,1995-08-16,true,763612951.857,11:16:57.2 +-1505036713,3556276413183099,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.642,1988-07-26,false,106185661.151,12:41:21.13 +-767715235,91284320167820784,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.642,1976-05-18,true,310108885.866,9:57:12.13 +1421516761,11124499466812068,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.643,1979-11-15,false,1024841956.34,19:55:16.22 +-400979485,91049043369925648,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.643,1961-10-07,false,1353178784.35,11:36:56.41 +1725014434,73185468755761344,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.643,1989-02-05,false,627645667.671,2:55:13.29 +-1877623364,54722169342361400,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.643,2007-06-27,true,772094901.902,16:16:49.45 +1012116882,19881723930536880,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.643,1950-08-26,true,1209364086.78,7:21:8.10 +-1897947437,79680930392449760,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.643,1978-05-26,false,83114438.5357,1:20:26.30 +-1126166359,44211143125754168,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.643,1996-10-21,true,1244822879.68,6:49:35.13 +1048724350,25288693894088764,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.644,1970-09-19,true,660940013.443,4:5:45.52 +1577609993,39296773922875464,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.644,1959-02-17,true,1360792784.73,6:17:2.52 +2053437451,44537675668280904,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.644,1963-02-20,false,125144362.323,13:32:1.24 +103989998,68736640473843552,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.644,1992-11-06,false,1301230430.42,15:55:50.54 +-590703673,42965427384129248,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.644,2005-01-12,true,855414234.84,10:41:42.35 +1906499897,1066124923451852,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.644,1992-06-13,true,447913881.184,17:1:40.22 +1372839077,1470579262426695,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.644,1994-01-14,true,541057160.216,1:17:47.10 +-394120983,25305854586950012,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.644,1982-09-11,true,40437008.2641,22:38:14.13 +-1436851350,12972702653250530,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.645,1959-11-22,true,51535714.6265,19:35:45.41 +-2117409515,35096065145485732,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.645,1960-01-16,false,1326553857.46,6:22:14.46 +-729370698,51711853933743288,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.645,1994-05-01,true,1217012252.65,12:9:56.40 +1586784418,88942597987353520,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.645,1961-06-01,true,785621211.82,3:32:8.53 +-1590135380,49810949242243240,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.645,1966-06-12,false,512373660.598,5:36:13.54 +-363972160,2981881370848768,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.645,1968-11-14,false,781358232.942,21:50:51.14 +1714823395,63724936568398840,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.645,1967-09-24,true,800486635.357,21:55:12.14 +-1995548857,89619754772560512,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.646,1951-07-23,true,1269946267.64,17:5:37.32 +852166199,41827821688835640,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.646,1978-01-14,true,822278038.154,19:46:2.8 +1305901864,81692104557694480,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.646,1987-06-21,false,944081043.496,11:2:57.13 +-303173515,50523715642451568,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.646,1998-07-16,false,208014125.688,22:44:52.29 +743639184,24171679268094548,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.646,1997-08-02,true,306273375.32,6:46:41.46 +1402767161,81814361504163552,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.646,1992-02-18,true,111922168.993,16:37:19.12 +-982655826,4080816140076390,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.646,2007-11-15,true,1317171127.06,16:13:15.26 +-93842170,9612110704624896,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.647,2010-06-13,false,724492659.57,2:35:50.49 +806337081,41972038852002048,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.647,2011-02-21,true,558926301.381,15:7:47.54 +-846893896,23301926071978672,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.647,1960-08-21,true,1159243577.43,16:44:28.41 +403556413,51729435833116408,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.647,1975-01-12,true,865625185.953,9:21:9.9 +-1725170601,82700039041478624,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.647,1987-06-02,true,1140458176.82,13:14:3.38 +-200792940,89532381456544512,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.647,2004-07-16,true,110512853.618,15:5:45.55 +1612483158,48453214740350176,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.647,1979-05-07,true,1192012849.08,6:54:35.7 +-2063924861,49411423371337352,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.648,1955-10-13,false,102230305.042,10:7:57.44 +2134587031,79858193112827584,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.648,1955-07-15,true,582647831.183,17:52:38.57 +1256660215,52187155301158096,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.648,2001-04-18,false,305532574.294,1:6:28.5 +1375412506,13833265001404590,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.648,1970-03-18,false,1380873984.08,21:3:28.39 +1980879062,34346108648306464,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.648,1957-08-06,true,715320891.441,6:18:1.32 +-2109848882,28878013170499596,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.648,1973-09-23,true,939822543.051,19:40:24.33 +-2083403336,47428361937884528,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.648,1993-10-04,true,832058031.437,1:33:36.38 +1193843548,10169566144644138,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.649,1989-07-04,true,424207848.283,17:11:47.35 +814035486,81592496044156544,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.649,1978-01-17,true,86490325.0711,20:8:13.5 +232778394,4509313831614976,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.649,1953-02-06,false,1166183794.77,15:21:40.1 +1715661045,55052366841337368,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.649,2013-04-27,false,88400727.3225,4:51:14.37 +-272723216,63012921680968880,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.649,1993-02-07,true,691161615.321,8:37:14.2 +1473146901,24710897788595384,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.649,1969-08-08,true,414886788.422,16:58:5.26 +981792830,35568041213522280,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.649,1982-09-07,false,749174195.644,2:27:7.2 +-853888880,24865476462908816,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.650,1999-10-06,true,309123573.865,4:14:46.7 +37882585,7137502796387768,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.650,1961-08-20,true,296141892.199,16:53:44.17 +-753053594,28099310784346316,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.650,2008-07-03,true,1163682386.73,20:40:17.36 +-1992938546,49389052860059784,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.650,1969-04-01,true,1317599824.1,18:16:36.38 +-2057095629,36575250986369712,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.650,1951-07-07,false,642544764.413,14:9:18.49 +1650233034,71288758006934536,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.650,1952-11-08,true,300812583.445,6:26:37.50 +-301255630,76853901920625280,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.650,1989-05-27,false,1166984721.4,21:41:15.40 +-396012738,74475803191026992,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.651,1956-02-01,true,1355527387.12,12:58:46.7 +-1866233267,81242495864848240,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.651,2012-09-06,true,1198773081.87,5:48:1.19 +-724914267,19034164283232920,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.651,1969-10-02,true,828888599.772,13:36:10.42 +383491592,48690107320657552,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.651,1984-07-16,false,687120572.119,12:33:11.4 +1484915319,89950439097497712,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.651,1986-01-20,false,1326843940.35,4:17:6.50 +348106358,80120093377142832,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.651,2000-04-17,true,794415639.584,6:45:28.57 +1802010120,91673352002923600,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.651,1952-02-04,true,549462782.795,3:27:30.43 +-467000414,45167142761723048,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.652,1992-02-26,false,255281622.807,9:11:14.38 +1039290058,6906786560141947,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.652,2013-07-27,false,523630320.082,6:40:29.15 +-230023310,75113987399270736,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.652,1969-01-02,false,1323243506.53,7:4:37.24 +482553708,88436095345704944,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.652,1965-02-08,false,298815711.6,8:11:19.39 +1322044015,90286465031363984,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.652,1992-02-23,false,1109766948.45,12:50:27.55 +1501268439,70968998838202800,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.652,1986-03-14,true,1245500054.55,16:52:42.10 +-1146011388,50883804411853376,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.652,1993-09-21,false,588738064.437,13:7:58.38 +318936211,38589670899201744,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.653,1977-07-17,true,1301044210.57,16:30:20.40 +-176140575,90288809932672448,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.653,1972-09-26,true,628486991.84,17:38:40.8 +-1724209825,34585779680552316,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.653,1970-11-08,true,1381199641.64,19:32:18.26 +-2032859108,24350201107892144,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.653,1994-06-18,true,280266022.451,15:18:53.58 +-1317201544,41609467054754568,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.653,2002-03-18,false,277515593.213,1:58:52.19 +1499564018,39757277355813800,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.653,1970-08-11,true,1273014630.53,4:28:4.36 +-1950827322,39200839334455992,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.653,1976-04-04,true,369611820.172,15:17:17.49 +1119586321,51588983754756240,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.653,2009-02-26,false,1283953464.66,13:27:3.47 +1493130025,92076597681132272,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.654,2007-04-05,true,1113933458.0,12:49:39.33 +1002739176,75096120351920272,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.654,1968-06-02,true,1194180444.93,7:56:57.47 +-2124334958,62514200695072056,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.654,1966-10-14,true,677913123.169,7:55:48.33 +89727957,11271530121758792,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.654,1952-05-16,false,646754010.403,16:19:30.47 +-1596248154,69596906961991,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.654,1982-10-27,true,1117572799.64,16:30:54.36 +1127521069,37819174686396368,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.654,1972-11-07,false,1056364565.37,7:57:26.16 +-152316061,43548102595835176,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.654,1992-11-06,false,938395950.935,13:26:3.9 +1399319720,76399802681762640,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.655,1991-11-27,true,128208557.683,1:47:4.38 +1155518524,14160230774137866,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.655,1981-08-04,false,1067354712.12,18:38:50.36 +-2139119758,80812269028393520,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.655,2007-11-04,true,1152951891.83,2:14:16.53 +1507832439,41774054407756112,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.655,1982-02-19,false,761017298.97,18:36:42.19 +-238317708,65523437232719576,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.655,1982-04-04,false,288692149.281,16:48:42.11 +1800334366,81282939449657472,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.655,1958-03-09,true,586812670.253,5:33:32.31 +-1994994813,88455809002755696,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.655,1951-08-13,false,408605829.837,14:48:52.56 +936930729,18939996092051704,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.656,1991-05-15,false,757587683.016,13:46:51.19 +1070659938,57675814139972160,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.656,1953-03-19,true,1297881338.21,10:29:5.39 +-1094147533,71421416008793184,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.656,1981-04-16,true,1025545446.97,9:20:26.7 +919556105,26353974776456672,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.656,2006-11-24,true,393965847.938,12:42:6.39 +2031802591,24587254033222432,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.656,1979-08-20,true,382633699.348,22:41:24.11 +372044603,28594662487812948,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.656,1973-05-17,false,620898549.059,5:34:40.25 +-680533931,85844602270482480,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.656,2004-04-18,false,841535960.88,6:46:50.26 +-1449479485,42089828377592872,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.657,1955-11-15,false,1307263380.18,17:14:22.17 +-882647192,13206588079740714,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.657,1983-07-24,true,289530284.621,20:6:34.13 +-1745950053,66210184717826448,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.657,1978-10-17,false,1242008446.7,16:30:30.40 +-1514158227,67097273416675216,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.657,1990-03-13,true,1402123413.38,12:31:55.16 +475441604,16236557649465784,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.657,1977-08-27,true,868095800.597,16:47:20.53 +1387890881,66071646856804712,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.657,1999-04-19,false,950247387.2,20:4:47.19 +-1268195466,32290511740081540,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.657,1998-01-16,false,354057136.429,18:32:55.37 +560919159,82940896845802160,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.658,1980-06-01,false,461312816.322,20:45:56.13 +1330896446,6139776269871370,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.658,1962-11-11,true,1015964782.81,6:21:24.34 +-1703329690,26163127656544644,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.658,1979-06-27,true,271935978.089,5:57:21.54 +-852736829,5783673332174428,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.658,2001-07-10,true,923501089.236,22:12:52.10 +-788925160,73402514825856592,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.658,1988-10-13,false,1426230239.59,12:49:57.8 +-1500753392,39910061335978832,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.658,1964-01-01,true,642483349.443,21:7:35.36 +1604856118,36258891227084456,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.658,1990-11-19,true,1157225764.03,8:31:8.47 +643916005,1237589977133721,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.659,1951-05-13,true,1188414917.42,4:34:18.39 +-18181700,66121733996120072,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.659,1974-11-25,false,353760801.15,15:17:29.1 +-1368440980,90664767113253248,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.659,1968-09-23,false,139190944.775,3:17:12.25 +-896146642,51788243179624304,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.659,1982-07-09,false,312705970.853,3:17:8.28 +-1179711977,86440708487171328,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.659,2005-11-15,false,58752319.8586,6:57:50.13 +-854462128,43842051494003880,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.659,1967-04-27,true,1016216955.53,19:4:16.27 +228081697,32202629572423772,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.659,1970-07-01,true,1081332578.19,2:27:23.11 +-221134655,66372446541573912,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.660,1975-10-12,false,537984104.216,3:43:57.35 +-1411664935,79072336738387728,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.660,1970-03-15,true,946209194.845,13:21:9.16 +-300354853,30975524377113896,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.660,2009-10-10,true,1031737230.13,15:27:3.8 +-883074129,71707576827061240,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.660,1967-03-11,false,312108851.062,4:1:29.52 +852119171,88154284327072528,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.660,1955-01-07,false,931057963.537,19:13:34.20 +-55344623,84155788386338384,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.660,1996-04-11,false,511446083.76,7:17:35.1 +414458220,62702707045060120,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.660,1964-09-21,true,865175797.614,21:48:56.39 +-1717453055,58309211399804776,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.661,1997-04-03,true,623181584.365,20:45:52.33 +-2091036445,69815886180248176,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.661,1984-09-15,true,1241822218.55,20:51:26.13 +845319042,22987082649546180,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.661,1970-02-06,true,264898977.032,4:54:54.29 +1977265032,56691952753187368,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.661,1957-08-25,false,1233974339.25,15:48:36.10 +690700802,49282481349657336,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.661,1970-06-21,true,1233522502.92,15:28:49.33 +380374207,24520908856970764,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.661,1958-02-18,true,544373293.87,22:3:39.43 +-1651842499,89085258170316336,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.661,1954-08-08,true,297863147.874,11:5:4.28 +-510738534,40890906137031416,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.661,1977-02-01,false,1251161737.12,6:43:28.29 +-1280941112,2991004739601725,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.662,1983-10-06,false,1430271119.48,2:12:5.43 +-1342511392,73191111571955536,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.662,2009-02-06,false,1399766100.34,2:15:8.57 +1830803916,44705058651761784,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.662,1974-04-27,true,1186373260.81,21:47:56.31 +-2102657055,12018816775404226,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.662,2012-08-07,false,499332854.767,7:22:35.41 +1241506469,15669799996425340,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.662,1989-05-11,false,278106291.276,14:3:25.10 +-14754011,13426088625335992,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.662,2010-11-02,true,1219516627.9,9:15:6.46 +1645884025,88578968408501520,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.662,1953-01-25,true,1294996783.48,18:56:32.38 +1076054791,9681699404931430,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.663,1961-10-10,false,1123048141.61,9:15:25.26 +817383767,7549783727379180,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.663,1995-11-03,true,771931883.258,12:57:7.21 +1679990679,24902286466304696,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.663,1984-07-19,true,552331610.054,15:53:55.20 +2102964340,84885580642834512,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.663,1966-06-11,true,544421515.503,18:26:4.10 +-50141505,78360980407864512,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.663,1958-03-14,false,758783367.205,11:5:46.33 +1756851336,59978671825597624,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.663,2000-01-06,true,234288690.014,22:13:19.21 +87507967,82370619772132480,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.663,1973-05-13,true,413511650.759,13:47:53.22 +185703305,60789749386438512,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.664,1983-10-11,true,348559108.11,7:28:31.19 +2093503960,2947298281453895,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.664,1993-03-22,false,968047678.672,5:33:51.38 +1007146917,82987309104865840,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.664,2006-04-13,true,1100061794.7,22:15:15.25 +-1698496663,80385591641673888,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.664,1952-05-20,true,832273222.497,13:26:7.38 +-1415348676,10063062501618688,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.664,1955-08-18,false,1405495744.73,21:6:45.43 +1365333840,24032522036465224,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.664,1998-01-16,false,1211924989.69,22:22:54.6 +-314501193,62526546219958048,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.664,2013-01-20,true,154160564.959,14:8:53.53 +-212260378,73256300631624032,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.665,2002-03-12,false,1050570461.73,5:54:56.39 +-1843460106,43972247333683000,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.665,1974-07-09,true,484734225.672,15:16:34.46 +691483100,61945618040851464,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.665,1968-05-09,false,723846779.744,4:53:52.19 +1436799203,25849749406569800,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.665,1975-04-25,false,1344075165.91,8:21:29.52 +71918921,6011866638816348,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.665,1956-11-02,true,1311737829.94,15:33:26.43 +-2145688547,26831364579455540,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.665,1952-06-03,false,377327145.42,22:42:49.28 +410066040,90634099534231680,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.665,1968-05-24,false,906841247.103,8:23:42.55 +-2094997330,40916657790163464,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.666,1952-04-16,true,292598891.247,22:39:53.3 +1652190981,36702053821526320,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.666,1977-02-11,false,1154058861.23,9:7:19.24 +-783345772,21292538648298936,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.666,2005-10-07,false,455696887.75,20:22:57.27 +-1590311012,52966189860456432,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.666,2009-04-13,false,117835961.135,6:8:21.37 +-1551209611,38055138146017160,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.666,1997-10-07,true,912935220.495,10:21:22.38 +2023217466,44674333496796240,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.666,1973-10-18,true,505737913.687,10:47:26.13 +967820511,5698032107132631,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.666,1954-11-02,true,312425148.775,2:16:45.37 +-1211610558,55290565727167152,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.667,2001-06-06,false,463085344.89,10:5:37.10 +234165098,88544499635964032,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.667,1973-08-09,false,653853030.151,4:39:44.8 +-1873669936,794039273756928,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.667,2013-10-01,true,1318420204.47,19:4:21.4 +-722068186,89595879019031328,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.667,2001-10-26,false,1131589462.02,11:16:45.42 +1540712724,34492224168392224,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.667,1964-09-25,true,248832275.064,16:50:17.9 +1686471135,88071919756409616,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.667,1974-10-05,true,156871172.779,1:31:35.8 +-1859288734,66662123105246120,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.667,1995-11-04,false,1212236392.93,16:25:6.46 +1515586116,1287903249845207,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.667,1953-06-09,true,482399836.36,4:19:26.5 +-1040468683,5530598156209480,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.668,1975-03-02,false,1060676320.5,15:47:30.33 +-1083328823,50815709477272168,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.668,2009-01-21,false,224020830.362,10:11:49.51 +-1410749539,25341536351696876,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.668,1972-06-09,false,1101280721.87,10:28:20.36 +-1463012975,52326028753110744,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.668,1955-11-11,false,809537747.111,8:22:2.43 +-1098008057,80207746390014800,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.668,1953-10-11,false,1000812120.32,21:52:56.25 +113046759,10361781900759338,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.668,1990-05-07,false,1153985017.26,11:56:36.21 +-341081723,75908760476811120,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.668,1981-08-04,true,687821788.113,6:11:14.29 +-2065614955,68228301821975064,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.669,1985-07-11,false,831678225.567,11:47:46.20 +-114516034,50624478175174072,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.669,2001-09-06,true,1026446314.81,4:22:14.9 +-2105500382,84691085181193504,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.669,1999-03-06,false,734948605.147,20:42:37.26 +-1987180700,1067129368106598,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.669,1985-06-11,true,1074900193.61,6:44:42.44 +-159923369,89302488039363936,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.669,1956-06-24,false,86516631.5635,21:25:9.24 +-464752896,18974896814789060,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.669,1993-09-24,false,60631285.5371,19:41:17.14 +-275500966,43571083302801656,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.669,1972-07-27,true,1078468294.69,18:38:11.43 +-1020032869,8611122746867722,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.670,1953-03-23,false,1070383280.11,12:21:27.42 +-1943123315,14529924557426688,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.670,1997-09-16,true,140944417.759,1:38:31.12 +209904003,3545705272820275,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.670,1972-03-26,true,153745024.545,16:12:51.11 +868389260,21682194049098588,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.670,1962-08-25,false,682090775.335,14:25:50.30 +1093992095,56550614692977584,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.670,1988-09-18,true,1113715134.09,3:33:38.6 +-552325563,2122575488813813,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.670,1966-01-15,false,40872583.0727,10:12:52.36 +1738566404,40370785418596800,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.670,1994-03-10,true,825938146.546,18:52:22.26 +-835977403,2830682118093977,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.671,1975-07-17,false,75869405.3937,11:20:45.32 +279696125,80437102920390960,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.671,2007-07-25,false,408740319.966,13:15:52.26 +-1526825618,71217834022869648,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.671,1955-05-13,true,363757559.484,13:45:53.35 +-131929156,82233298363509792,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.671,1990-06-11,true,601750226.793,8:28:22.21 +1231375976,78445141799762640,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.671,1956-11-07,true,947510875.881,14:36:32.12 +98762854,19488945964187988,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.671,1958-02-01,true,1214476168.37,12:45:38.14 +-1844641257,27222097837834620,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.671,2003-08-18,true,980142938.855,6:37:28.10 +-122689749,78761278976635344,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.672,2000-05-02,false,922075180.209,10:35:36.17 +-897864633,30531616954165340,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.672,1986-01-07,true,92363113.2547,14:28:12.57 +1761790818,23049307104332676,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.672,1976-02-05,false,1384753527.89,2:23:36.14 +384627945,75203763593224352,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.672,1976-05-14,true,1082308524.8,11:50:9.21 +2088786689,29569266044899492,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.672,1993-06-19,true,1276091358.72,19:58:9.51 +-372651515,55525052863585896,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.672,1993-08-03,true,826533361.588,5:22:9.21 +218071437,42248626638792456,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.672,2000-01-08,true,267070687.026,11:4:49.56 +-19464894,65450834884062936,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.673,2001-09-08,true,992211160.128,4:50:25.24 +-339635527,14006574119175946,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.673,1989-06-03,true,1349295329.92,15:12:55.39 +1995704876,22463399583230424,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.673,1997-10-20,true,369028940.348,21:33:32.4 +-1668290320,45991112086491200,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.673,2005-06-04,false,626325357.407,6:32:53.31 +-1242869555,1641151154080819,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.673,1950-09-02,false,620726000.417,19:6:28.41 +1745698837,53453441503400520,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.673,1952-09-23,false,641810117.177,22:53:38.33 +-1749842746,59352503101509632,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.673,1988-08-14,false,661714587.18,16:15:18.6 +1066509601,16161338868416676,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.674,1965-01-25,false,426877799.592,15:22:4.56 +-1622008966,43866550548806320,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.674,1952-08-05,true,596298581.238,19:55:6.55 +210627460,26724328097049488,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.674,1955-07-12,true,63661138.4844,11:36:10.34 +-616589962,60821479728190848,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.674,1987-10-14,false,1262141986.07,21:7:5.48 +-1984127921,17865418796569570,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.674,2008-08-14,false,1313877317.56,9:21:57.34 +-1157090611,44672104367713528,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.674,1992-10-23,true,589935401.358,14:31:19.50 +-1291827969,60726972084377768,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.674,1979-05-07,true,51836101.2795,21:29:47.16 +1270001461,40436573315086840,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.674,2011-04-10,false,21255519.73,9:36:5.16 +-1868598065,53885148849123752,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.675,1981-05-23,false,714739788.669,19:11:51.32 +-1461033399,34802069341472144,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.675,1987-05-23,false,1001222992.91,21:25:29.19 +1873491102,10647003320239166,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.675,2007-01-25,true,253629754.468,16:12:29.21 +-298589495,70079903075487856,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.675,1981-02-15,true,801436640.065,8:39:32.48 +-619567705,29572557677834608,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.675,1984-10-10,true,509958199.164,6:9:5.50 +-570818361,30317060352250276,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.675,1967-01-06,false,1422434301.67,2:51:43.41 +411476676,19835752796034180,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.675,1956-10-06,false,1241197433.96,9:51:36.33 +-1481479693,50315117254789360,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.676,2012-05-01,false,251511284.449,17:17:42.3 +-534156228,736762743671285,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.676,1968-04-26,false,306234339.386,10:8:2.34 +-1094296493,13905813047357604,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.676,1968-02-14,true,1065279889.76,1:6:56.54 +-1976329172,17870080284242616,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.676,2003-09-25,true,506777957.162,20:38:11.20 +-993270536,8954124099998495,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.676,1970-01-23,false,15739806.7021,20:56:1.27 +-1846688067,56571494387759616,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.676,1954-09-01,true,311291074.062,8:14:37.30 +903464596,62924115613194208,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.676,1992-02-22,false,242372227.097,13:32:8.46 +-16337924,68283814413912240,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.677,1981-05-04,true,542579493.304,6:30:35.30 +-918044842,33400304297995532,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.677,1971-03-17,false,529510095.161,5:25:11.40 +1232751056,82999255189888784,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.677,1968-07-26,true,1225180629.42,9:56:44.3 +-1177282053,18459294122629344,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.677,1987-11-03,false,1343063914.59,16:3:52.33 +445215417,3175006744668385,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.677,1963-10-10,false,884951863.817,1:47:3.5 +1297788107,24901187757325148,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.677,1985-11-23,false,1234034987.56,11:39:24.40 +-1429284330,41215285350795176,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.677,2003-03-23,true,72151022.7807,22:34:21.55 +36496650,74455522780630784,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.678,1978-11-16,false,1090453262.95,21:49:20.10 +-477490409,65348128347959256,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.678,1957-07-27,true,82152900.3314,11:53:52.34 +-748693156,51823715636463496,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.678,2002-03-20,true,999926940.806,11:6:46.14 +-1719913739,77203051783487872,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.678,1961-06-12,true,1384473748.4,9:9:45.56 +1835797860,91298364982464080,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.678,1967-03-21,false,348611523.705,18:43:47.7 +659409419,60389071398611152,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.678,1973-08-18,false,130476108.375,22:27:56.15 +338158645,7855001177053450,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.678,1979-06-21,false,1174050484.19,3:8:19.2 +-1871990409,2000813262254540,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.679,1978-03-24,true,457842708.476,13:15:13.12 +-285181712,91257681303035296,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.679,1987-02-10,true,900801605.547,4:4:9.5 +-55711162,41673477836082488,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.679,1978-05-02,false,927893813.063,1:9:6.45 +2000536427,52928264492425696,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.679,1965-05-10,true,1072262648.91,13:15:42.52 +576392226,80426494879093360,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.679,1970-08-14,true,528504971.036,2:47:33.6 +1462796989,87441636571840112,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.679,1987-05-15,true,536942264.518,22:58:22.35 +812591996,71900813810597488,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.679,1968-03-22,false,910895589.743,18:7:52.30 +1502105789,123048755244943,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.680,1997-04-10,false,890885657.306,22:19:27.49 +-974846532,78635940101906800,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.680,2005-05-26,false,134539989.024,2:47:57.31 +912212378,72927747659051072,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.680,1997-11-02,false,980545995.973,17:38:9.32 +1750674024,47804374640783672,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.680,2000-09-27,false,406119236.033,11:4:1.58 +-378086034,15931849512343932,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.680,1959-08-23,true,796138908.384,8:44:14.44 +-852077238,18055392557181860,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.680,1994-07-14,false,1425605941.27,5:50:16.40 +1586866373,20812414095824496,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.680,2005-01-03,false,404154613.59,13:13:17.31 +-162276773,51513215572437912,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.681,1953-04-12,true,328459591.025,18:35:28.57 +-35752652,59341082847348320,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.681,1962-08-24,true,125584876.06,13:4:43.44 +1290151748,14460330702490584,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.681,1969-11-27,true,656595797.594,16:16:12.4 +-1436584807,57961508895813600,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.681,1959-01-17,true,1212808213.07,17:11:51.56 +-1147940313,34875148941736744,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.681,1958-09-02,true,1014997295.58,17:12:4.41 +1600038613,67559714521579848,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.681,1950-03-25,false,54291085.3691,4:26:8.55 +414435207,16608225440189686,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.681,1977-05-05,false,1155479498.86,8:20:40.54 +618672645,53698372076722432,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.682,1966-03-27,true,910742633.05,10:24:36.8 +-443295744,23986100524877488,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.682,1967-10-21,true,1126321202.64,22:46:4.23 +904220739,20861232454526884,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.682,1975-10-14,false,660755524.326,13:44:5.19 +-980439509,20950758201550704,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.682,1987-09-27,true,704980782.163,4:14:6.13 +-823663650,29364681193571740,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.682,1951-10-04,true,1326168245.18,14:29:44.15 +-831708350,33984344815066480,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.682,1994-11-14,false,206181304.678,7:8:47.32 +-1915904609,74467065710476784,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.682,1965-02-21,false,852287244.015,17:27:4.35 +-1566199960,54888574784384184,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.682,1985-05-25,true,739497910.128,19:40:44.30 +1868504749,19065981927699824,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.683,1987-02-04,true,1219986840.87,5:12:34.37 +1962868925,74087096302904336,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.683,1986-02-10,true,168060594.088,2:36:49.19 +1408345271,16323304740284110,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.683,1986-05-18,true,562575988.347,18:2:6.42 +-1556708805,40568625997856192,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.683,2011-05-25,true,176513904.082,17:49:52.35 +-847245357,86359406681199488,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.683,1982-09-26,true,768135253.072,15:54:55.42 +-1162162176,70318831177262736,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.683,1966-08-21,false,271703521.597,13:24:22.46 +-1702587054,71733800951704416,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.683,1988-04-24,true,299956553.319,9:5:25.54 +1132977891,34248685630282856,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.684,1991-11-27,true,11201607.7008,1:31:11.12 +101991090,71865398408580920,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.684,1980-06-18,true,217849496.559,11:10:24.17 +-1322097671,80232610172315664,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.684,1961-09-08,false,687763368.411,4:57:27.27 +719651287,37797778471266296,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.684,1976-04-06,true,28238489.0352,18:33:33.47 +-1053649065,62494272824912416,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.684,1990-02-26,true,691026807.841,12:18:19.32 +-356217386,2162982590496061,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.684,1998-11-20,true,601997123.83,2:30:12.26 +1834162460,72376813656586048,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.684,1978-08-21,true,98011119.4496,7:47:46.25 +17141649,75989210687914784,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.685,1989-01-13,false,945951683.54,1:4:12.15 +-1947862111,71061201636993544,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.685,1964-02-10,true,1220577842.27,7:46:35.54 +2015144950,24659562158297836,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.685,1992-02-20,true,594946293.537,18:24:51.27 +-1347417025,16318950947843256,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.685,1991-05-19,true,890231066.072,22:53:36.16 +-82427512,69964539073315392,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.685,1990-03-21,false,830001015.084,5:1:54.20 +-276075765,87666216655018944,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.685,2009-08-05,true,80339344.5647,12:52:7.12 +1003188396,86328597565421984,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.685,1978-05-08,false,158065765.364,3:40:50.51 +1664160114,20444018791185900,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.686,2004-10-25,false,175199336.418,12:34:54.55 +796694205,36114560624611792,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.686,1996-07-03,false,952461101.14,14:33:32.39 +1142499374,55746235363566360,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.686,1951-09-12,false,642593408.424,20:45:9.28 +172707685,3699071840045353,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.686,1965-04-27,true,647075799.841,11:1:10.20 +-493262963,61888870098699160,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.686,1979-07-20,true,939027667.236,8:42:37.7 +1950019749,22614398669438628,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.686,1973-04-23,true,854136508.85,9:43:53.19 +1365307331,58543201413662688,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.686,1981-01-19,true,452824692.626,6:43:20.45 +1319137584,68718681864785432,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.686,1954-03-07,true,565783387.344,3:53:21.23 +1165124316,78879263110722144,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.687,1970-02-27,true,37629234.7545,8:55:16.55 +1107380972,23136619832925964,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.687,2001-09-06,true,1285141866.45,9:27:26.49 +-1531515665,22004496988437300,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.687,1964-03-16,false,1154868232.19,22:50:2.4 +408634605,70684086995803960,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.687,1995-05-05,true,1063542867.07,14:30:42.24 +-37320047,2629031037376420,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.687,2007-10-09,true,255879230.353,12:54:41.7 +-1499087518,60703410132205344,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.687,1988-06-17,false,1107648087.54,6:42:7.52 +-1420896389,86236837618136640,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.687,1996-02-18,true,729937286.795,18:8:20.16 +-1748040077,36009852448630560,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.688,1963-05-20,false,68669648.5796,9:45:30.5 +-675081998,7611822531243213,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.688,1976-08-02,true,681454790.134,11:6:51.35 +1529423499,45029791886211480,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.688,1981-10-08,false,634178935.325,20:22:35.12 +1828471635,37574267376472184,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.688,1991-02-14,true,195913309.682,22:10:25.24 +-1383802578,71626285263139688,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.688,1959-02-18,true,81493099.689,9:55:46.16 +791338895,71716760854475848,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.688,1964-10-24,true,258104549.838,11:10:18.23 +-268197871,10872027707283568,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.688,1950-09-04,false,701069786.212,14:41:33.14 +1833547907,66757563597737584,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.689,2005-09-27,false,1111511290.08,8:39:4.39 +-1478123182,3950625683011338,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.689,1959-03-13,false,1168188611.71,10:3:42.16 +-1764251657,18500780404993476,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.689,1970-09-06,true,716429838.304,1:11:51.42 +-563729810,26127544114125988,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.689,1954-11-24,false,534847234.229,9:16:24.10 +-1804780742,42762165864143944,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.689,1955-05-03,true,829914575.879,8:38:17.35 +1623455981,4284141785313853,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.689,1951-09-05,false,914779739.138,5:42:27.57 +-1288268016,75714737758222240,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.689,2003-02-23,true,718732827.75,21:40:45.39 +1782169520,5205613025283093,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.690,1984-02-22,false,771038154.311,21:8:17.37 +-383757072,66485887430860584,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.690,1986-09-03,false,335424145.866,15:32:4.13 +1008540282,65692428282702048,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.690,1993-11-10,true,1432910117.55,10:36:14.46 +-1956883149,37586433687144896,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.690,1995-02-03,true,1279159438.15,4:14:28.55 +2091008332,22259385961091676,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.690,1979-10-11,false,55045835.2523,10:17:34.44 +-928355295,53163781135685496,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.690,2006-11-27,true,1290307414.36,9:19:33.29 +1105207143,60487573939951280,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.690,1958-04-22,false,846672083.701,4:53:31.54 +212348766,47424586051747112,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.691,1969-11-10,true,1046629412.72,4:17:34.44 +621767935,3806584959539272,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.691,1982-07-07,false,1344064524.11,22:41:22.17 +1078872517,58847383884230952,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.691,1989-09-25,true,1135128748.65,13:35:26.45 +1807043117,13007666930795992,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.691,1971-07-15,false,87160502.2896,13:13:57.41 +1879153000,64305488244556856,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.691,1956-07-18,false,889771357.913,5:25:31.44 +-909109668,86919245307709344,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.691,2001-09-04,true,1366404697.2,14:54:51.13 +1127685585,48208108162973760,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.691,2002-08-24,false,434196373.449,9:15:36.34 +931112232,49927171015367176,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.692,1973-07-11,true,411384801.38,20:18:56.1 +188715489,28863453001146420,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.692,1995-01-15,true,92031858.1001,9:12:56.15 +805038824,78535712016914960,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.692,1988-03-04,false,809319353.16,21:42:38.36 +1242582814,5633786081120768,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.692,2009-06-11,false,1050827733.15,20:3:2.51 +712268663,76484429602355456,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.692,1965-01-10,true,260778471.825,9:13:10.17 +324830367,13117756333549364,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.692,1966-01-07,true,138388198.929,2:19:57.1 +-1739704146,74439688479747808,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.693,1983-08-23,true,218828710.044,10:26:24.2 +959966928,60005341848051016,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.693,1965-01-08,false,197113177.712,5:57:35.56 +2112794423,68976258912361032,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.693,1966-07-07,true,29359348.9481,9:24:28.37 +-1273158350,22235864638487676,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.693,1994-08-02,false,173304266.059,7:18:15.17 +-2001917764,53883000182128624,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.693,1975-05-19,true,539225496.272,9:9:27.10 +302764232,1122785824473262,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.693,1990-03-05,true,865482588.182,18:17:18.32 +869906218,79857142993922960,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.693,1992-04-14,true,525117208.179,17:15:7.48 +-854239102,48222458269087288,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.693,1991-02-06,true,681992215.846,7:45:2.12 +-277264551,38405551173233760,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.694,1961-08-26,false,1288207019.84,13:19:14.4 +-1039097215,58801108108550480,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.694,1956-10-04,true,451132832.87,11:34:50.29 +-897690847,40930948250397536,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.694,1954-05-09,false,37856794.3292,22:53:23.47 +1007326253,69350962721557616,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.694,2013-02-26,false,885956673.838,20:4:57.22 +969648371,48396540357491920,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.694,1965-11-16,false,277889922.013,12:53:4.15 +1580814218,59740893728826880,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.694,2006-04-23,false,256152475.206,3:18:43.30 +-1899277157,87117813943860848,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.694,1989-03-09,false,81182741.7112,10:29:13.43 +638512891,32034754826509088,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.695,1982-01-09,true,1108616060.75,8:3:26.4 +-396562372,21648749373429792,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.695,1975-02-27,true,632554418.915,3:16:52.21 +-1773356839,73660540798745472,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.695,1984-01-23,false,19147069.76,8:36:27.22 +-66078765,13211616316455076,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.695,2013-04-14,false,1049270168.65,20:24:14.15 +-911201254,38227853008392992,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.695,1980-06-22,true,758699856.809,9:42:11.27 +-134648776,26329239993485024,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.695,1953-07-04,false,1136724943.54,16:47:3.54 +2119531081,10671589744389980,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.695,2000-08-01,true,1148260375.0,3:17:52.30 +-463113086,84619419816792608,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.696,2010-07-18,false,1125482316.87,5:37:34.1 +955103497,50034589674133288,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.696,1970-11-10,true,1284216978.63,4:30:14.49 +37864798,16819805569634868,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.696,1995-08-11,true,730373130.549,15:24:22.22 +-770887004,39413611725443952,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.696,1978-01-14,false,554068929.957,7:25:48.35 +-800072654,16705307585062534,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.696,1980-01-26,false,645035279.428,17:46:44.23 +-529275901,9188367210009610,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.696,1961-10-10,false,656979172.356,11:14:33.47 +391808322,86091797626232464,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.696,2011-08-14,false,1120492698.66,3:42:48.30 +-619680702,2525579050760970,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.697,2013-08-05,true,246736285.165,14:34:34.28 +1472267490,35532205797401500,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.697,2011-02-19,true,204866046.527,8:35:21.57 +1306585264,14426006800416266,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.697,1979-03-14,true,1162075030.11,4:50:6.6 +1697264168,236503314252349,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.697,1983-10-24,true,1144783300.72,22:53:32.32 +-620268162,20942623582634784,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.697,1984-07-24,true,1006167191.7,2:39:58.5 +-254805165,81236469132916752,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.697,1957-01-18,false,302120616.099,22:3:31.8 +-162182840,69703148284051160,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.697,2009-02-06,false,877084372.072,7:54:23.47 +-169114351,83451091527474832,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.698,1995-07-04,true,4823164.43726,12:56:53.17 +-425455956,61829546079727688,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.698,1996-04-19,true,203426896.146,2:40:18.29 +296412559,59567642028161128,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.698,1986-04-26,false,174131103.739,6:26:24.29 +2137062025,64197188771118904,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.698,2008-04-12,false,592539142.559,9:49:54.17 +-440024585,89872200050025584,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.698,1956-09-19,true,427958360.205,18:42:17.39 +142167482,71172311317228984,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.698,1991-01-22,false,962768560.25,3:39:24.14 +799832583,57228935573815096,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.698,1994-03-18,true,799195351.825,2:27:43.58 +669008457,16711835339593216,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.698,1955-03-02,false,224901846.503,20:12:35.8 +-871209019,90185768121333312,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.699,1951-04-01,false,354456085.643,18:52:32.19 +921782999,75178870583402080,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.699,2009-03-05,false,1006819101.5,6:43:35.31 +-353960682,82772470830670416,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.699,1996-09-16,true,911872260.641,15:27:45.28 +1452556833,9911625428159284,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.699,2004-09-12,true,1139028142.31,18:31:19.11 +98738829,17827938939210824,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.699,1959-03-06,false,744117990.26,2:53:13.54 +1268338434,58980634125431440,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.699,1961-07-13,false,51476936.8578,2:10:21.41 +-521719337,36369829021758800,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.699,1951-06-05,true,1089654924.42,6:53:27.18 +1505438617,24228859508779684,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.700,2012-05-04,true,296365844.787,17:51:43.6 +-19137734,72717707940949168,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.700,2012-07-21,true,1247318154.76,1:17:14.35 +491384321,75485899066691632,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.700,1967-08-27,false,1133964446.86,10:9:51.6 +-1221190268,60947924484202864,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.700,1971-03-05,false,962284290.505,14:29:54.28 +-1936474161,19281646456137492,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.700,2004-06-02,false,986566688.308,2:26:52.27 +2100243690,69470942506678152,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.700,2006-06-06,false,1223656652.38,5:50:17.11 +1651983736,86191480408999104,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.700,1979-08-22,false,531161261.21,2:55:42.33 +-1657601715,48835795137821032,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.701,1985-06-01,true,328902851.521,15:26:25.14 +-1882892105,59387953272233880,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.701,1985-02-01,true,305812506.943,8:27:7.55 +-160040235,41260337578859376,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.701,2004-03-07,false,791586492.265,7:1:17.52 +1760292972,78829101281398800,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.701,2013-11-06,false,418461794.054,7:35:51.55 +-908509,70549322599107800,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.701,2013-10-25,true,1146727590.9,13:26:10.49 +-501540858,81757342386016368,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.701,1970-01-04,true,1275199719.97,10:13:18.27 +563021486,85713081482170080,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.701,1988-06-23,true,1063112833.19,22:47:32.37 +1072341704,40833998903983432,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.701,1981-06-10,false,918099574.601,17:30:29.25 +-2086693342,73898999915172256,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.702,2004-08-15,false,378553518.167,4:14:16.22 +766510289,57624211553219288,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.702,1952-06-19,false,1272972281.5,18:20:47.4 +1471967627,44383325105255120,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.702,1987-10-26,true,930978446.276,10:50:36.10 +715483958,58231708043483720,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.702,1951-09-13,false,555616366.645,20:11:31.20 +832519060,22624572561017200,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.702,1994-06-14,false,1420453216.73,4:54:1.33 +588306844,78803199942184688,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.702,1977-05-24,false,1172684874.04,5:7:51.41 +159627826,71071776809300760,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.702,1976-10-21,true,426581093.808,21:18:30.29 +764085062,26546244061729772,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.702,1999-07-12,true,599875304.704,6:20:3.31 +1346900560,92042993506569600,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.703,1989-06-13,false,833561756.683,5:49:7.5 +-1597524373,86457498275056144,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.703,1965-05-01,true,55380229.0444,6:32:29.15 +1444698136,14679621860609966,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.703,1984-04-08,false,1114828071.52,21:9:56.11 +1548843152,60232505807627960,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.703,1969-05-11,false,227630307.135,21:7:4.28 +2126780350,74424289733912656,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.703,1957-06-06,true,526722054.58,22:9:1.6 +-1988448559,87212023862474416,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.703,1998-03-26,false,1307179120.16,7:47:46.41 +1910686945,84117036987429488,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.703,2004-08-20,true,207290362.462,2:18:21.54 +-1070874167,8951555198699326,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.704,2013-09-04,false,142192747.877,21:24:37.17 +-1071717752,79375734865177328,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.704,1963-09-22,false,1031177468.33,6:58:16.20 +1038157793,1248874466523832,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.704,2001-07-16,true,797977903.397,15:58:57.4 +1953582122,43315881504101568,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.704,2010-02-11,false,1240203367.17,6:8:34.33 +1086684462,42857945664543928,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.704,1953-11-14,false,1320298728.54,8:48:11.40 +464699276,42354501255739496,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.704,1989-03-06,false,1166793665.23,7:35:40.22 +136406148,3800017531923978,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.704,1966-08-23,false,87864214.1903,1:3:54.57 +-118692911,34517718438498324,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.704,1979-03-25,false,350019095.567,6:47:51.32 +-544171697,60169089680209160,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.704,1974-05-10,true,1219398244.14,6:10:54.51 +-463258820,8906861940637215,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.705,1969-05-08,true,1397352372.86,5:34:45.1 +-1687839219,30018857108358072,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.705,1970-07-15,false,1266907664.8,11:37:3.11 +-525714443,15725328994077778,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.705,1959-07-25,false,703080316.389,8:23:20.16 +-2131309379,27152927794367120,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.705,1984-02-14,false,473067644.629,2:29:14.11 +-1625725751,55347073442348168,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.705,1966-05-22,false,378673344.355,7:48:9.9 +-1999760511,13865776022745456,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.705,1960-02-23,true,849896937.835,12:5:19.50 +1479130918,45799328652043336,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.705,1968-02-27,true,37699687.8835,12:41:35.26 +904338635,59050526669493816,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.705,1984-07-27,false,1039264710.78,19:9:24.47 +-1689389799,78512869126835312,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.706,1999-09-03,false,1152680839.4,8:47:7.31 +-74122976,18235433913850008,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.706,1996-10-16,true,592804573.71,21:25:17.54 +-1621671412,23177488773728728,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.706,1976-06-03,false,661385733.059,4:14:17.8 +-1227933824,82041118063984688,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.706,1971-04-10,true,848613955.8,16:6:29.48 +-746605597,33057964170093980,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.706,1951-03-09,false,1238954533.82,20:16:30.5 +1518323220,5235752230777549,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.706,1985-09-10,false,166256712.815,3:49:28.19 +135351617,10529337464009062,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.706,1951-09-05,false,1092589046.87,21:29:4.58 +1614404273,56202421175690600,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.706,1966-07-18,false,475019212.166,19:26:28.23 +-1813315312,23766415211100336,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.707,1982-07-11,false,571837688.334,21:37:40.10 +294353250,57258507496221512,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.707,1995-04-18,true,601778045.073,7:44:42.40 +718907727,24348817776938364,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.707,1990-08-04,false,1030877538.63,3:34:6.8 +-601732021,48176598716133816,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.707,2000-07-13,true,474106821.554,4:27:50.15 +1847983251,90006849432804320,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.707,1975-08-05,true,1008590689.32,16:8:38.48 +631275306,37600587372199088,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.707,1997-08-06,true,247981431.631,18:23:15.22 +-1817209489,17499257987633500,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.707,1958-09-06,true,1380803693.98,10:1:19.37 +1118839165,18477646215886224,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.707,1982-09-09,false,1042093685.06,5:52:3.47 +1792264293,60256852104507216,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.707,2010-08-27,false,23950089.5856,16:45:10.41 +-2094536939,21072718221269516,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.708,1963-01-23,true,965038502.01,6:4:15.5 +-357918165,37647590337369888,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.708,2009-09-08,true,1188821345.08,20:44:48.56 +-1926915894,77342224596400032,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.708,1988-11-09,true,585543828.483,20:5:16.51 +-989257235,25906651813716676,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.708,1967-10-20,false,252771527.592,14:8:42.21 +-1137361322,32120370792204584,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.708,2002-05-22,false,1417778104.54,13:46:9.34 +719303279,15376350005850696,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.708,1979-05-18,true,256758530.541,7:23:49.58 +1317974413,79798998386946464,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.708,1951-09-05,false,917286593.521,21:28:28.1 +23627721,37381210561782888,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.708,1971-07-01,false,408964619.911,1:46:26.29 +1798800184,40558045005781456,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.709,2010-05-14,true,993165119.818,15:13:12.31 +1294993483,21748877207326260,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.709,1980-05-23,false,1211850315.57,15:6:2.9 +1527927025,90570701149100784,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.709,2000-10-02,true,737740714.869,16:55:52.22 +-1475889369,32251828844958352,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.709,1959-06-18,true,1321544984.21,11:50:23.11 +1424487488,64673580457544000,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.709,1971-05-16,false,332770918.717,12:45:9.58 +2028148203,31805636640317216,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.709,1991-11-04,true,618850026.788,14:47:49.46 +451398379,28414031338380608,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.709,1987-03-23,false,105526851.578,18:38:25.19 +-1301503645,6343854416036588,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.709,2007-06-19,false,998393578.066,3:11:40.41 +-926153584,6558726014331679,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.710,1952-10-22,false,678455971.807,16:32:12.31 +901366946,25765825819798404,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.710,1961-01-26,true,133383613.591,16:49:5.52 +-2013652972,2134379540135536,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.710,1999-09-24,false,93744108.2563,12:8:1.19 +-2133342271,1168022907154462,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.710,1983-02-09,false,402622258.505,15:45:9.45 +-796155824,45226823309779184,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.710,1974-01-25,false,108393368.24,7:9:23.24 +1788088877,15068471093960490,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.710,1957-06-16,false,727633968.902,18:10:50.19 +-1434732781,69995026554557664,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.710,1990-08-10,true,91938781.6523,4:51:1.27 +1508321249,64647726890032864,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.711,1979-02-16,false,720860743.675,15:58:25.22 +-1290180724,43176521913207224,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.711,1966-04-07,false,852682063.914,17:55:28.28 +-778866319,22848048104315064,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.711,2000-01-27,false,257447747.793,6:6:2.16 +312065298,26604725966222100,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.711,1958-05-23,false,270723859.304,13:31:42.44 +-695004625,36794381138362072,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.711,1991-05-01,false,321053833.351,7:24:14.2 +-666463906,80526115464236992,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.711,2000-03-03,false,645721421.089,16:55:16.22 +-1114016947,84127913535047568,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.711,1964-08-19,true,384534043.198,16:52:42.50 +-192191300,3274486871901552,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.711,1980-01-10,false,149078133.997,8:34:20.52 +1070459231,35092406662596252,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.712,1996-06-25,true,1199138012.65,19:43:36.51 +-1247511841,55520720937279808,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.712,2006-04-12,true,1402119396.66,9:45:1.28 +-366059339,70964808203980400,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.712,1977-05-16,true,830011033.361,20:18:58.32 +-1247393598,87148273331606176,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.712,1994-02-23,true,1078770767.06,8:20:34.54 +357008953,63043257145415576,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.712,2013-10-10,false,1216813825.55,19:34:17.2 +-934334918,83334163761359776,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.712,1964-09-25,false,1156944490.11,1:29:30.52 +-1280372993,30237905626561548,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.712,2012-04-01,true,314040581.377,5:2:35.54 +2917101,48926470085700872,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.713,1985-08-14,false,1272625871.67,16:5:27.27 +-1028981889,3363614190335252,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.713,1954-03-15,true,1168546254.99,4:5:17.11 +-447733292,62614696086806304,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.713,2004-06-23,true,30576367.0011,3:7:31.18 +-1154584158,30453116742460548,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.713,1984-04-01,true,760607756.11,10:23:41.7 +47682632,3924538620365855,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.713,1988-03-07,true,827944971.241,22:3:41.53 +-2074627616,3568506919884984,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.713,1985-09-01,true,271192121.854,10:50:8.31 +-1924641069,48704769025177464,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.713,1988-04-26,false,11028535.4041,2:14:55.38 +-1838797398,88392320707064368,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.713,1954-09-06,true,79513000.5589,6:5:39.6 +1632547222,3936271545818634,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.714,1962-04-25,false,1068970255.7,21:33:11.12 +-973879132,2255014143215114,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.714,1967-01-20,false,1224013489.88,17:37:14.40 +-1645722163,19802932160545036,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.714,1991-01-25,false,1122574187.48,4:7:20.56 +-1966970994,45243534091917200,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.714,1970-07-12,false,325537048.007,15:44:53.14 +-1961674329,58367258266921224,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.714,1964-02-07,true,1359807536.36,17:11:12.34 +-1593430963,90997640121254528,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.714,1989-01-05,false,202449065.718,12:52:16.56 +-1830157271,41812856704738472,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.714,1967-10-11,false,670456077.247,21:51:7.12 +854981896,4380721002024745,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.715,1972-08-02,true,1285478983.13,11:22:31.50 +1372890564,21235680295086328,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.715,1974-11-12,true,930994793.034,17:25:52.45 +-1199701018,43294744496694424,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.715,1999-09-06,true,568907887.229,22:33:21.47 +1630748409,71651085018814976,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.715,1974-09-06,false,402159561.169,6:28:27.48 +-403739181,2736265109659955,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.715,1964-08-14,false,389226710.247,19:28:4.46 +-513570485,84438915270491984,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.715,2005-05-26,false,421830298.183,11:2:51.10 +-1815595908,86894969221515808,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.715,2005-10-10,false,927843955.895,2:18:6.33 +496575510,13680757614229340,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.715,1989-05-08,false,1044753876.9,13:44:52.23 +1229264669,68076953765293928,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.716,1974-11-09,true,577977368.229,4:53:27.58 +1588872345,55335488733343008,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.716,1967-08-13,false,363494889.23,3:49:11.51 +-1496722627,30910923396625920,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.716,1993-06-14,true,790756382.356,18:30:49.4 +482169861,22502531719355056,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.716,1975-03-07,true,764045164.246,1:58:20.56 +-1143254724,56835582864099760,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.716,2009-05-17,false,1007583760.24,10:34:57.7 +603286874,80297403899009520,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.716,1989-02-03,false,436277417.07,4:9:29.36 +1221683510,24106996533877996,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.716,2006-01-11,false,986132934.613,1:56:47.3 +1708647352,4009137526242263,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.717,2011-07-04,false,743497466.008,16:27:9.26 +-1073117782,16381657918656328,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.717,2013-02-07,false,347672517.983,5:40:6.40 +364466826,14256459022465086,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.717,1955-06-26,false,1327007703.08,7:29:44.1 +-656686907,23748723837574624,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.717,1990-02-25,false,173076362.126,1:40:6.5 +-804697441,82913307428450272,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.717,1973-04-10,false,733871180.203,18:53:30.8 +1061640606,2961699592677949,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.717,2003-07-23,false,1048142886.32,6:35:36.12 +-2116120581,55665693738314704,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.717,1962-09-15,false,1368038609.94,7:44:27.7 +-2104885917,74368502306164064,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.718,1987-02-14,false,450369385.062,15:52:37.42 +573387805,40382702957325576,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.718,1998-09-22,false,285134.031315,14:35:46.31 +1083388079,18552924532214520,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.718,1998-03-22,true,98975852.1597,21:32:55.3 +-887711039,71154442054590320,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.718,1970-10-23,true,311894349.044,15:11:7.15 +773257945,20586678782720316,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.718,2009-11-24,false,1081450045.48,16:19:2.53 +1179709261,53102016956540000,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.718,1965-01-14,true,868157045.316,12:14:47.8 +-856620923,24529280149036872,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.718,1982-08-18,true,723883724.503,18:48:5.39 +-213590877,3488005103075942,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.719,1958-02-21,true,522678524.224,11:23:8.39 +1164775246,21961475590091164,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.719,1993-01-21,true,331296900.535,3:46:2.40 +-919778246,29527539857277072,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.719,1962-05-09,false,469450745.846,4:34:34.15 +187712005,72499957344159392,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.719,1980-08-21,false,1043039493.94,8:41:15.16 +-1532948586,78006101613236672,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.719,1996-01-22,true,1146790218.61,2:1:37.5 +94850104,15339643484129812,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.719,1989-06-10,false,135479300.514,9:2:51.11 +-420954911,80595233101504816,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.719,1983-09-08,false,554572577.264,20:2:55.21 +835191423,58436400878396344,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.719,1960-05-03,false,76563270.207,1:29:41.58 +-88015788,44685874694891264,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.720,1982-05-15,false,1369705916.83,13:51:30.53 +-838966259,9944001617624514,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.720,2012-04-14,true,144651720.627,4:36:34.26 +2134598640,53691541695844576,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.720,1962-08-10,false,143083099.273,1:25:3.16 +2057183345,36651625449145648,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.720,1999-11-23,false,394737267.26,14:51:10.45 +-861114036,18180525728737188,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.720,1992-07-07,false,965240157.098,1:23:28.27 +131617787,84532528583502976,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.720,1950-03-05,true,954054350.953,3:48:8.9 +-516803164,50785071268410472,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.721,2008-03-10,true,419662167.892,18:10:57.39 +-106678811,45032280444520528,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.721,1962-04-11,true,814158869.354,19:51:38.18 +1536662088,77292314764709856,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.721,1993-10-07,true,314242871.851,19:37:51.5 +821996723,25292170908569416,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.721,2008-05-27,true,626350596.397,19:3:55.25 +1642913533,55717847666358784,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.721,1996-05-14,false,213609996.731,6:56:18.21 +-296355167,11143564360293684,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.721,1968-08-13,false,578197743.113,22:18:41.35 +-89380822,54313402631549256,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.721,2001-07-21,true,1172118195.81,10:9:12.28 +-1755606940,7371705122641459,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.722,1957-03-06,false,628704231.71,19:21:19.38 +570260306,87393600032575728,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.722,2012-10-05,false,842567551.629,8:27:4.6 +-503762004,47495747557895032,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.722,1979-09-19,false,1028350168.05,19:51:21.19 +2106426341,54054975975087248,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.722,2008-03-20,false,1148912553.36,11:37:4.58 +-1368956587,51733053261606712,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.722,1974-02-21,false,1358455707.94,16:56:10.49 +-1025667707,50887900958682136,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.722,2005-07-25,true,8400821.55481,14:15:25.4 +932731181,51949368725771896,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.722,2007-03-14,false,1409136186.53,16:36:3.11 +-442950703,38184882231545224,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.723,2007-04-12,true,396536109.609,2:50:36.30 +137190370,34957847422528728,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.723,1952-11-08,false,1374021973.56,6:3:17.30 +284254552,78417704956393296,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.723,2011-09-20,false,560521866.278,8:5:14.45 +428480980,36005173845380068,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.723,1955-10-27,true,420127481.851,15:42:10.10 +216189522,60052379669266760,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.723,2005-02-11,true,1207501193.63,4:53:46.30 +-1487022523,23798314100459092,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.723,2007-04-14,false,637355571.184,1:15:37.15 +-1722494714,12535228630784706,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.723,1954-03-27,true,1243687269.97,17:57:11.22 +1182328057,54334172800125112,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.723,1951-09-20,true,618560393.666,3:15:37.39 +-894367176,40482863136382648,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.724,1962-04-05,true,137098450.339,5:9:56.17 +-313318630,5478324960555960,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.724,1953-03-17,false,203439391.049,17:37:57.6 +1771314695,39016119122700864,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.724,1994-04-14,true,470221126.483,6:13:45.3 +-906813636,56137379274691288,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.724,1998-08-22,true,448187227.238,11:32:17.58 +42659212,51971556293157856,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.724,1972-11-02,true,380217514.862,21:31:4.54 +-910911787,11171605162123028,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.724,2008-09-12,true,1270579718.39,8:28:57.44 +1428374592,14282783128119204,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.724,1967-06-07,true,1373075089.72,14:26:19.18 +914302327,23892153912621384,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.725,2013-06-08,false,915741369.195,9:51:45.29 +-601462555,17797373271511910,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.725,1982-03-25,true,1155875617.56,9:16:14.47 +1394969151,62267770306290672,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.725,2006-07-15,true,710065962.631,19:13:1.25 +99723574,53662393780005888,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.725,1952-10-03,false,1025961638.12,16:21:27.56 +-1096621491,50290559117325112,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.725,1977-09-19,false,1229128638.09,20:32:28.43 +-1991011838,84102793879140224,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.725,1952-04-20,false,1168157654.45,2:1:38.15 +-1046063642,48879902313421848,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.725,1974-02-11,true,1146773946.13,19:51:32.6 +1838457828,66780618360162616,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.726,1978-06-17,true,1190734862.17,5:49:45.5 +-954555665,3036699999425966,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.726,1973-11-10,true,1363597032.36,19:37:41.10 +-728614991,11311958021110518,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.726,1982-06-16,false,1008172278.0,5:10:43.34 +495535341,46203987944983800,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.726,1961-07-04,false,1339797507.28,8:57:17.13 +1293116189,84642797211565328,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.726,1968-10-06,false,388292218.027,20:38:30.10 +-818563186,76845371035778464,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.726,1963-02-16,true,630356366.203,10:58:19.8 +40284806,59689068214094152,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.726,1984-03-23,false,399124001.11,17:31:34.40 +236426519,4824655029800325,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.727,2011-03-13,true,175359862.697,21:53:44.20 +302599695,69732893711085160,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.727,2013-04-07,false,113570532.556,9:13:23.55 +-763923817,29304080988772968,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.727,2005-05-01,true,1163795551.28,22:25:58.43 +-1485066250,63685072829368392,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.727,2005-07-16,true,156803961.8,6:10:29.24 +1558795777,82268835644802208,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.727,1956-11-07,true,227735404.654,20:44:19.13 +1220489151,40923788670314528,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.727,1968-05-18,true,395295371.985,14:1:8.15 +-1540233711,53072936183333264,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.727,2008-07-04,false,1036364540.15,6:49:13.30 +1923448448,83219797780246032,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.728,2006-08-19,false,783316170.982,8:10:28.1 +1208830920,58072705449448656,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.728,1990-04-21,true,271509407.668,2:15:41.43 +-970293031,78957776619506576,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.728,1990-02-04,false,1149480193.35,11:13:52.47 +1038120317,63003763491004160,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.728,1971-02-14,true,328812884.738,13:44:26.50 +1942003627,37258110391784896,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.728,1964-04-17,false,846066174.986,9:35:21.33 +-5876672,61769568486903488,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.728,1998-10-16,true,192523460.336,8:27:45.18 +-1366308251,28001010107436400,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.728,1982-05-17,false,928207709.512,14:26:53.21 +237818234,31103099052022620,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.729,1958-01-20,false,1130651001.16,15:31:2.22 +-1771002498,66036692965403280,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.729,1986-01-22,true,1145644733.95,14:36:4.28 +-940533094,90235672864256080,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.729,2008-04-12,true,854164703.293,12:38:4.45 +-1373345864,73299156855637872,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.729,1997-09-22,false,1380380112.75,15:52:8.2 +1765526442,74993439284434720,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.729,1974-09-20,true,1221497056.57,16:55:40.36 +-1416222586,38225691432824880,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.729,1975-08-15,false,1326906194.08,17:22:15.9 +-797830770,55751956809449168,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.729,2012-10-14,true,126640715.18,5:12:14.31 +-1732857557,84159468340677968,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.730,2009-05-11,true,427305844.986,6:12:14.3 +836820430,78686179060850672,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.730,1957-06-21,false,293121884.708,18:7:39.44 +2137006522,70637328764558296,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.730,2013-01-13,false,1304303779.03,12:2:13.3 +-1389869516,26230155136150404,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.730,1960-05-06,false,1115210560.95,5:22:43.58 +-617067194,61610413466561024,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.730,2009-10-03,false,161392213.022,15:12:25.51 +-124228049,49709827722957248,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.730,1957-05-25,false,494214078.355,17:56:4.49 +978475870,62799647675537336,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.730,1954-07-27,false,52636678.1496,3:48:2.35 +922328614,6452362198189568,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.731,1991-04-18,true,732289857.979,22:42:40.5 +943586425,25743954101941916,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.731,1999-05-07,false,314050451.394,18:16:56.42 +1352693952,17070853177775606,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.731,1984-11-02,false,1062604066.75,15:47:12.39 +-1317605628,14593004932803184,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.731,1992-02-01,false,1137640309.6,12:56:34.5 +-448393221,56071796904024632,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.731,1974-10-20,true,710790394.669,5:19:31.3 +1351896112,3929672405527982,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.731,1972-07-10,false,571803163.001,8:7:7.30 +-108777111,90820972225047920,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.731,1969-09-19,false,415883753.409,21:23:45.15 +1803522895,22184250873903584,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.732,1992-05-19,false,206858440.793,5:29:1.29 +-864718880,43340731951269960,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.732,1999-11-02,false,1286956148.12,18:2:51.1 +-351283641,73708007515882864,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.732,1973-02-20,false,1206414745.14,2:39:7.12 +813611458,44253266622539112,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.732,1977-01-15,false,835662551.055,3:18:45.44 +-1901252291,20598288260891292,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.732,1982-09-07,true,997654607.929,14:19:17.21 +29092004,67134333564654224,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.732,2011-09-23,false,240595322.846,19:36:20.21 +-1110607228,52192353124773032,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.732,2013-11-21,false,780345778.904,16:51:4.50 +-964466068,7553699988439624,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.733,2003-04-18,true,36818139.3214,10:25:41.6 +-1145235866,59011615082441288,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.733,2011-10-01,true,1021582428.93,13:16:48.25 +1162418881,71003840610889128,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.733,1985-07-21,true,432300270.794,17:13:35.46 +-1219136698,59355599325801984,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.733,1991-02-04,false,646825754.234,15:15:29.22 +54170306,45410404256144048,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.733,2000-03-20,true,267232804.058,12:54:9.30 +-983589891,86972453011594256,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.733,2002-01-22,true,125525845.647,20:38:10.28 +1438782605,90805493800200032,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.733,1992-01-17,false,10465798.826,19:39:15.49 +294937697,84372661871679456,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.734,1952-02-04,true,1106937826.15,6:41:35.15 +1410425522,66081014748342328,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.734,1979-08-15,false,1214881741.62,20:22:18.27 +1671442901,52000797613002616,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.734,2011-07-19,true,700310580.344,22:15:4.19 +-2032099625,8852768837478666,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.734,1956-02-21,false,98581260.0381,9:36:14.12 +399798889,8542412290235730,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.734,1968-03-06,true,222697761.445,3:56:40.21 +-476755164,22107281069171232,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.734,1952-09-03,false,436987088.633,19:17:18.28 +67676424,74961642968133168,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.734,1976-10-13,true,1325148987.97,2:56:19.1 +-210419132,83450916381808896,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.735,2004-10-06,true,1152790269.43,20:38:40.48 +-1521190302,74128806493123664,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.735,1987-03-23,true,1020328453.68,13:20:47.19 +746759417,41461000687234528,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.735,1981-05-23,false,386742238.604,1:32:33.22 +-1043670707,86667022478574384,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.735,1995-01-23,true,888726122.376,4:4:23.47 +942386348,34768879876101940,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.735,2010-08-01,false,1349341736.61,11:47:58.40 +-1820501050,8199240300621066,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.735,1969-10-13,true,588752407.143,17:49:24.30 +1439331845,53641119441422696,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.735,1963-05-04,false,1328953209.45,21:55:38.19 +337359877,35280703258699184,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.736,1972-04-26,true,960786113.24,15:34:57.18 +587058854,83945986277138608,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.736,1985-02-02,false,232849342.702,19:49:31.43 +-1463340979,90024940648230816,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.736,1954-09-25,true,1250854103.13,11:4:45.41 +-11857492,61949490889001952,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.736,1978-09-21,true,237803652.573,8:33:58.9 +-1778352765,71192985462891688,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.736,2012-11-02,true,573959442.473,16:53:27.4 +-1887255320,48070325657553504,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.736,1977-09-01,true,1421694414.83,14:57:23.44 +1321289670,51942946468189184,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.736,1995-10-25,true,1385506711.73,8:9:2.14 +174194202,71622783819693744,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.737,1977-06-08,false,872418867.383,9:24:46.38 +49499255,77517030722451840,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.737,1991-10-02,false,1122443021.52,12:17:56.28 +1636951732,55747050523584408,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.737,2000-03-21,false,998712908.546,7:22:58.46 +75248641,61301214914380792,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.737,1962-10-21,true,1328100603.07,1:31:28.43 +-1845280522,48833701935806168,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.737,1978-10-23,false,1280916024.74,8:26:37.56 +1059595271,87482679026867072,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.737,1990-02-11,true,1045492384.73,3:13:15.35 +-661535798,85543725409323280,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.737,1988-01-02,false,1160193012.8,4:43:2.56 +9501980,13214012623364034,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.737,1969-02-19,false,66339833.0721,20:3:43.45 +712782615,68989874590703024,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.738,1996-08-25,true,582268128.709,19:48:53.47 +1772122489,83119412718692720,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.738,1996-11-13,true,535106015.073,11:42:15.37 +975757474,49449419558853640,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.738,1977-11-02,false,1182521622.35,10:42:6.47 +232802731,3021487880043069,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.738,1986-02-07,false,33331224.2481,14:3:39.47 +630469904,35580975316314780,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.738,1992-10-22,true,1161158787.79,21:23:48.44 +317999831,74890681334553696,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.738,2005-09-15,false,1148122033.4,5:44:30.54 +234712642,34351974478049436,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.738,1989-03-06,true,1214489574.35,10:48:13.10 +-1637271488,33250040724664964,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.738,1984-01-10,true,371430012.504,17:35:3.4 +-1458243544,27269977431799696,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.739,2004-08-26,false,971570964.076,2:44:40.30 +1557042731,54686913943170400,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.739,2002-10-01,false,851038544.911,13:33:13.57 +-396024211,26175623113664012,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.739,1952-06-14,false,729471619.216,18:27:20.9 +1041225733,15809261954190940,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.739,1974-07-19,false,425961474.459,11:1:43.38 +1754381079,836215565420441,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.739,1968-01-25,true,1423322910.83,4:16:4.53 +1191353474,32005296590894848,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.739,1953-05-26,true,1370399712.19,6:10:42.45 +1084424768,71706457967929424,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.739,1969-03-08,true,198892368.453,15:4:46.20 +-1900653864,6111919027552471,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.739,1970-03-13,false,112714070.361,1:42:9.9 +-1167446321,5189099594640220,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.740,1974-03-18,true,952831115.037,15:23:30.48 +358989593,84227567172383568,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.740,2002-04-06,false,894286978.923,4:32:23.25 +73722620,27197619372761192,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.740,1959-06-01,false,1351288765.14,15:14:45.27 +-400455641,3980284684487874,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.740,1964-04-03,true,634681110.858,12:19:42.35 +-2087934931,46046204813965920,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.740,1964-05-20,true,1076091985.41,18:18:39.57 +-1645885371,71285262402661664,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.740,1955-05-22,false,638006353.214,6:16:42.24 +-177834401,56386691362405440,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.740,1976-05-01,true,222284941.048,11:51:8.54 +1860311739,51463827768794552,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.741,1972-05-20,false,571873660.622,7:2:36.23 +83108701,24075994233020540,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.741,1991-02-22,false,1283073284.43,6:32:30.7 +500329814,79760161442371904,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.741,1985-07-13,true,569648056.205,21:50:2.13 +-421968076,4628193430803927,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.741,2009-10-10,true,1313598342.6,9:57:36.27 +1402175935,55253890895111672,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.741,2009-03-04,true,1156061810.97,18:39:23.11 +961685686,74658344686652240,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.741,1979-02-27,true,1190097521.64,12:20:4.9 +-1288469312,21485874239658580,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.741,1972-11-06,true,474665893.711,22:2:14.28 +1304324371,53587691970334320,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.742,2010-01-07,true,281843245.968,12:29:39.3 +1442233141,67389581481377984,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.742,1983-03-06,false,191791014.515,15:38:17.3 +1087708905,6571524235766293,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.742,1997-10-19,true,83394242.967,19:38:48.53 +1556293460,747598848015851,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.742,1952-11-27,false,1108974902.2,15:13:5.22 +77781730,27561957399404524,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.742,2010-03-06,true,1089892499.38,20:54:32.1 +-2037236718,23218722419426376,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.742,1965-06-13,false,313940887.548,17:34:31.33 +1018621158,63573347258289512,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.742,1992-05-01,false,497629092.641,20:11:2.42 +-319531214,22161675605422952,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.743,2013-11-01,false,707231562.727,5:52:18.35 +737142667,56965918210401032,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.743,2005-06-10,false,15413771.4626,16:19:13.17 +1323571433,61878811690246624,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.743,1961-08-12,true,970741122.197,15:22:48.26 +-103445882,42356274856123248,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.743,1994-11-15,true,889371477.334,3:17:33.16 +525268523,76016307512341888,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.743,1950-11-20,false,633191165.76,21:21:21.8 +1981571044,77240376459050480,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.743,1992-09-16,false,667911898.283,15:48:8.26 +-1667563325,39050039171244496,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.743,1958-02-17,false,1262180850.12,15:40:15.43 +-86993207,5349004981935329,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.744,2012-06-12,true,1117353391.78,16:47:12.17 +1916264432,23029126702677936,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.744,2012-07-04,true,864193290.723,17:4:51.25 +-1232613550,43047438049263480,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.744,1960-06-12,false,590783179.103,22:46:40.35 +215999015,65806903385901152,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.744,1967-08-03,false,1092190866.4,1:30:42.6 +-1776125056,12415104043004232,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.744,1961-08-07,true,348632735.512,21:18:29.4 +-919275941,79340256200791744,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.744,1955-10-04,false,623732692.589,5:1:41.40 +479328236,87950887206229120,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.744,1989-11-05,false,576547289.92,16:52:52.47 +-676538707,79580361502411856,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.745,1979-05-05,false,946740486.814,5:53:37.34 +-1527527148,45742886421297624,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.745,1992-02-26,false,690451220.226,15:15:12.45 +-275480873,61044946373465928,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.745,1992-08-13,false,1300817418.97,18:54:10.12 +1674130488,73742372489681120,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.745,1982-09-08,false,1019792574.51,20:46:48.49 +-1352916270,8229750516290826,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.745,2007-08-08,false,922036389.567,13:20:4.21 +-242830644,27026628376616284,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.745,1962-08-20,false,713917301.706,11:40:9.22 +1172586653,34856128119790860,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.745,2009-03-17,false,149898070.389,16:27:21.7 +-365264744,1455333096405514,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.745,2012-05-13,false,973123808.778,22:49:9.53 +-179394562,84681245779160448,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.746,1973-01-14,false,718447462.535,8:53:12.38 +-1853807003,34336435205783296,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.746,1981-05-04,true,1402722283.43,8:18:27.42 +-668093618,53828528024181840,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.746,2008-05-26,false,971749349.859,8:58:58.27 +1983620354,451905572832112,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.746,2000-05-14,false,574504174.176,6:20:25.43 +649271570,6710715155597148,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.746,1976-05-17,false,785139416.588,14:35:6.19 +1176475869,84626003016005040,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.746,1999-02-21,true,228727057.813,7:48:56.18 +-150796351,48162643370342872,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.746,1976-09-19,false,115037807.677,22:9:19.57 +70104137,17906602990781912,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.747,1955-10-12,false,1102519090.73,11:44:3.11 +-258852228,60008146193775616,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.747,2005-06-07,true,489508020.492,19:48:27.9 +1024894993,6846115665099920,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.747,1986-06-16,true,22831310.4981,15:13:36.20 +-663641053,66773496319516800,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.747,2000-01-10,false,187148856.48,5:18:53.40 +840651659,55527591038537944,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.747,1957-06-09,false,906593943.097,7:16:33.29 +924322786,17524266253051494,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.747,1987-07-02,true,38060577.4456,4:30:19.35 +2120387778,65183935515038136,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.747,1996-11-01,true,972476652.891,19:56:37.32 +463092756,60126217681359000,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.748,1973-03-10,true,455710778.912,13:7:16.3 +-1749097701,18801425158400144,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.748,1962-10-22,true,1340226092.93,20:1:32.15 +-830627642,47434693500006176,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.748,1971-03-23,true,80746010.9168,5:26:38.36 +1182496279,32436705718762672,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.748,1983-04-08,false,649572967.904,8:46:57.7 +765888122,64149697676512584,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.748,1957-05-21,false,212441244.004,19:11:52.21 +387994386,9476223615786906,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.748,2013-03-01,true,838939370.739,10:55:14.48 +-703972453,43961631007515496,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.748,2013-03-12,true,353411620.375,21:22:26.43 +-968014224,53520688136888832,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.749,1982-06-18,false,320266967.187,19:36:45.10 +-75495016,17504261893779784,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.749,1999-08-06,false,1082663529.52,15:58:45.8 +967722718,32992431657995348,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.749,1982-05-22,true,745822140.331,16:51:51.1 +-740659326,69499675141985816,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.749,1970-03-05,false,277405568.843,11:57:14.29 +1965496734,71221877885547072,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.749,1993-04-07,true,934275257.779,19:57:56.36 +-1053098459,46154536226680120,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.749,2010-04-22,false,985801210.86,3:15:18.31 +246140500,88908230569502320,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.749,1960-10-03,true,32097331.1649,12:54:10.21 +1384205411,59300049583106448,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.750,1998-11-22,false,341935040.547,3:36:35.18 +705620170,63277406553439448,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.750,1953-08-14,true,946563531.74,14:42:25.34 +543430316,49351118261507072,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.750,2010-05-02,true,950149836.953,17:39:17.22 +-936242709,29778254655528776,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.750,2007-11-11,false,651710623.49,6:33:45.33 +-1645936967,27419190596942048,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.750,2005-02-06,false,1300515239.46,6:11:37.14 +-1995366630,29270514398019904,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.750,1974-10-11,false,921631126.697,11:6:52.11 +1440278776,52586462077049704,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.750,1962-11-05,false,541052368.194,10:30:3.57 +-1324690493,6113650219416791,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.751,1961-03-10,false,34856631.3741,16:19:35.2 +158349238,38526679677817440,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.751,1972-05-14,false,131593785.681,1:1:26.23 +-585308844,33931549281907272,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.751,1992-04-19,false,646981781.876,18:26:7.20 +645038957,41073209421008520,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.751,1971-06-21,false,79503124.3142,10:46:51.26 +-57882901,24335964832872060,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.751,1993-10-23,false,204552300.611,3:24:38.33 +-1392105100,4388282404414832,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.751,1956-05-21,true,880487750.086,13:41:41.3 +-1362429739,21783148586497700,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.751,2008-04-03,false,1127493394.5,3:58:52.12 +-1964110482,40003583687000720,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.752,1969-05-07,false,707225529.353,2:29:40.35 +-217892534,25076275720198136,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.752,2008-01-01,true,229892007.812,12:44:46.7 +-1265220704,28007848694066504,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.752,1967-03-14,false,1266231760.65,13:2:23.6 +1409657872,87595476068748624,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.752,2012-03-11,true,852783106.115,11:5:54.21 +1636166451,69880933909384928,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.752,1957-07-04,true,804463770.763,9:7:19.3 +2138801922,83301058147782224,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.752,1968-08-05,false,657037818.096,6:52:50.37 +-794093843,82237118907405168,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.752,1997-03-03,true,238364372.517,11:14:13.53 +940374670,88662731844375968,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.753,1961-07-10,false,3609374.05141,16:41:8.56 +1772743034,25035832208721296,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.753,1985-03-27,true,828607298.378,16:42:42.46 +1154465721,68725427278399960,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.753,1966-01-12,true,1375914444.35,16:15:58.44 +861698635,91733704543071360,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.753,1961-04-21,false,1267478440.77,11:13:40.45 +-1929405889,78854699600471488,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.753,1970-02-10,true,1166365875.17,13:46:55.36 +-235182931,61052157738120808,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.753,1984-08-19,false,1335268099.54,9:4:33.33 +-102600393,46802889085747528,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.753,1957-10-19,true,79658090.6359,21:55:19.48 +-841920440,68439565738900960,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.754,1991-06-04,false,1019049431.07,13:1:51.14 +-1359155721,23198055070280532,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.754,1984-11-14,false,42034280.4875,13:32:4.41 +-238564444,86696938149738240,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.754,1952-08-13,false,1156316064.77,20:9:50.14 +-1298946211,21942965558755604,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.754,1995-05-19,false,101710978.323,7:32:6.3 +-984684555,69715720624895800,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.754,1987-10-22,true,1243794555.34,18:9:21.41 +402491960,4140699085015071,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.754,2000-04-16,false,849468580.01,4:15:2.44 +160540657,74722945804868976,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.754,1951-05-14,true,661531527.101,3:1:42.12 +319927176,10448048965585480,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.755,1983-09-09,false,1204216462.96,10:3:19.6 +1457494752,82671234888053376,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.755,1963-08-17,true,40485395.2529,20:39:28.46 +1124041670,39206993395225592,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.755,2006-07-07,true,850327908.119,11:31:40.10 +1926270760,4550397449816894,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.755,1983-07-15,true,1118150449.11,7:51:43.41 +926689499,8945331329756181,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.755,1958-09-20,true,1331969750.31,17:17:32.4 +2108352482,3693006490742569,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.755,1962-04-19,false,82545025.5943,2:7:20.25 +-1804701043,52209930873880648,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.755,2012-05-01,false,1180649074.23,22:49:14.45 +2052945500,31409322454364784,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.756,1967-05-10,true,520351151.897,19:21:12.25 +2096670820,58986783007814248,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.756,1980-06-20,true,222649506.787,7:27:23.40 +1165426134,40039475149832992,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.756,1962-03-27,true,1210914828.62,22:7:52.45 +-2019164459,78010780854838768,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.756,1996-06-27,true,35083608.9214,1:25:39.6 +-1295883785,57191926532596904,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.756,1976-08-06,false,1279209039.44,4:22:21.35 +1608429693,15771987875616758,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.756,1972-08-01,true,1056026561.93,19:47:16.45 +-219252107,42963789376007528,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.756,1976-04-05,true,952152793.621,3:32:56.11 +-849234897,91237284532960640,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.757,1966-04-16,false,1332681643.17,13:43:38.18 +-2096084716,74541484713234352,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.757,1980-07-14,false,130483108.274,16:16:6.50 +-512165550,12249377938165504,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.757,1999-04-07,true,938093462.701,6:3:1.55 +802240187,58829206011068000,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.757,1955-04-18,false,1111664796.7,9:30:11.7 +-937838327,12799757395866112,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.757,2006-07-16,true,675489788.017,16:44:4.30 +1221415478,44174278462739608,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.757,2007-04-12,false,1359012208.63,18:51:28.16 +214651629,6451644303842110,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.757,2006-08-03,false,514102168.14,22:31:50.39 +1971265167,63058362575498864,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.758,1991-10-11,true,898102879.196,11:49:29.46 +-1926401560,84330692345379344,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.758,1980-07-18,false,9417265.84046,12:57:8.32 +-1059329982,42458966959449552,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.758,1977-08-07,true,1213908035.95,17:55:40.18 +-1219093385,23050739836913900,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.758,1968-05-20,false,1349413431.18,10:50:48.46 +1105551299,42896205282826248,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.758,1975-05-03,false,813638557.667,18:30:42.6 +-1620515696,63749376583120064,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.758,1995-04-08,true,1065672955.56,8:20:2.38 +-124179347,74850428041135024,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.758,1985-10-10,true,407203829.279,16:56:27.23 +426449191,44959784395037912,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.758,2007-07-23,true,823365024.122,8:15:33.39 +1059523572,89693509834232672,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.759,1993-03-21,true,1283732689.35,15:15:19.2 +-299437137,26669319476956664,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.759,1958-09-19,false,206254831.966,22:16:11.27 +1344483114,45618459280137480,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.759,1956-09-01,true,568054031.241,1:40:9.15 +1531413409,17629503098119004,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.759,1994-05-26,true,1423256591.25,15:7:20.32 +847720930,10930173981868370,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.759,1970-08-26,false,304533809.323,4:22:56.11 +143203478,37351602568182024,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.759,1975-08-13,true,1144387499.17,10:37:38.3 +2047109865,20980006247940648,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.759,1995-09-27,true,652208993.252,17:33:51.36 +-666549380,34757907944239524,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.760,2003-01-19,true,1093783176.33,14:51:26.45 +846121525,74337965619723616,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.760,1957-04-27,false,234329061.1,12:31:14.26 +520970663,53002532045307448,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.760,1992-07-09,true,1161130553.82,18:21:45.12 +-935195917,25959832550478388,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.760,1970-06-21,false,76325403.0089,5:31:35.9 +-207219000,41451885185490768,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.760,2005-05-05,true,76129468.2194,8:8:52.57 +250490510,82378736490753200,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.760,1973-05-24,false,84738949.4,16:12:39.41 +-1757556464,79419436525047648,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.760,2004-04-03,false,1271708399.92,6:2:56.31 +961675122,86337804217472240,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.761,1972-09-18,false,231825821.224,20:20:45.13 +-9109952,55768157683159552,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.761,1959-09-21,true,263889249.61,2:42:4.44 +285639158,91287999617148016,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.761,1958-05-04,true,995672903.633,8:37:34.30 +1583782472,24388493988141784,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.761,1953-04-02,true,1127072762.94,3:33:3.24 +1900224034,81671845517329392,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.761,1957-05-12,false,614754738.222,22:8:57.46 +-582835687,8125321996226458,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.761,1984-07-15,false,291509389.593,10:44:2.24 +335027107,36428413486829056,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.761,2009-02-24,true,652169763.559,14:21:9.43 +2133618282,62127517816734432,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.762,1997-03-04,true,460905970.067,17:52:33.30 +-471300409,47315503109684552,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.762,1997-10-08,false,569912033.379,22:11:39.35 +-1817632963,71047250889395232,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.762,1981-06-15,false,381740887.846,1:14:31.49 +-1940364837,90203814400537360,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.762,1983-11-18,true,1118229982.14,16:43:31.38 +1117457425,10567490987686114,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.762,1950-08-09,true,818823932.031,14:37:41.42 +1126138951,9379512661075292,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.762,1957-09-13,true,1415471464.64,2:37:25.53 +667931938,46957599076124584,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.762,1950-11-06,true,166343359.139,8:23:46.6 +1891978293,48504175885343312,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.763,1997-10-21,true,1281275589.89,14:34:52.2 +744767700,58523321276375872,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.763,1982-07-21,true,293596139.81,4:54:44.37 +1693736428,36949166827539784,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.763,1961-08-21,false,405155308.739,3:48:15.6 +520967195,4737249063137249,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.763,1979-03-17,false,870449198.081,2:24:42.39 +890235145,85328946071166080,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.763,1979-01-03,true,1174616167.97,21:4:45.57 +157235475,14720718677017948,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.763,1962-02-05,true,522115032.856,17:40:31.31 +-1419228019,81618346691937312,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.763,2010-09-15,false,425960968.995,1:17:12.29 +584437453,76832065667316768,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.764,1984-07-01,false,1331370893.95,16:28:29.27 +364240201,14403411751585526,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.764,1985-02-26,false,575096434.897,7:25:53.51 +-1858602275,71100115832219352,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.764,1994-07-25,true,24279112.2442,18:57:35.5 +804194013,78039163952004208,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.764,1985-04-27,false,914487858.447,1:14:42.8 +123753144,50236147038039288,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.764,1998-08-02,true,741625209.839,14:17:40.43 +-1071684925,24237170201359340,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.809,1988-11-24,true,125769123.911,18:23:57.8 +1148142130,71971201833333448,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.809,1963-05-14,true,1134385009.05,13:3:22.32 +1671683545,56305523809577392,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.809,1986-05-19,true,1233649668.26,5:16:12.10 +-1197239719,28786618165255896,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.809,2011-08-13,false,1239128506.9,18:33:30.49 +-1614983248,42230341175921960,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.810,1956-03-26,false,1149138289.9,17:20:47.2 +1345625576,40650256894063712,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.810,2006-01-22,false,996942620.624,13:57:32.5 +2142484608,87098825307770912,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.810,1950-11-05,false,1085938090.04,6:49:9.28 +1512239687,71659779968398104,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.810,1983-10-07,false,482491782.55,19:16:54.54 +-1783247926,13279786137159630,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.856,1990-11-05,true,1371275915.56,10:17:54.54 +1566302638,37996752713544304,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.856,1954-08-05,false,690895510.647,8:15:34.50 +911844577,40967860689806232,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.857,1998-07-22,false,756204069.171,1:27:51.47 +451165202,90351828068189200,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.857,1967-07-19,true,1056454221.05,13:20:21.57 +1061766644,14428028859285822,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.857,2000-01-04,true,712633725.962,16:48:46.16 +65694539,69788953687495680,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.905,1994-09-27,true,1382016255.64,2:58:14.48 +426770287,74596386112691152,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.905,1962-08-04,true,851267205.268,3:42:31.1 +-1850439342,21166589285256376,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.905,1958-09-19,false,1146575286.45,11:3:17.28 +-1858442358,78414239593232176,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.906,1977-11-15,true,679179269.176,15:17:37.26 +753370126,18017603070399108,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.906,1962-11-02,true,31197897.2137,17:17:36.27 +52460594,84176162391498320,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.906,2004-09-20,true,56713283.9669,3:57:26.33 +-1176280897,920066305323540,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.906,1992-05-01,true,763118255.674,4:22:55.9 +-1431647019,54399380610501104,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.907,2004-04-12,true,309411418.738,8:49:2.50 +-2089926867,20109261525806048,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.907,2009-10-25,true,214189509.748,3:24:49.34 +44532224,4459882091699108,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.907,1986-04-07,false,648289136.994,7:39:1.47 +-306120578,22995108485346508,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.907,1970-01-05,false,1021137742.79,14:43:13.29 +-767756135,75020388566258832,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.908,1999-04-03,true,645334824.779,6:44:32.46 +-888025775,41439312718835168,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.908,1984-03-03,true,500241068.091,5:5:56.22 +1546238112,13647161776664504,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.908,1980-11-06,false,1344856103.71,10:42:49.26 +1367088960,88662207300809552,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.908,1959-11-12,false,808623575.824,21:47:46.46 +271133941,41226532466815568,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.909,2010-06-13,true,247460402.875,11:12:25.47 +-1584595649,56111412980391808,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.909,1979-04-08,true,367732647.208,13:51:24.18 +162172903,1001660068787251,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.909,1996-10-12,false,213321643.607,1:55:41.49 +658379553,20269814315292152,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.909,2012-05-19,false,799506820.795,6:14:48.37 +-952994583,62675765504360816,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.909,1991-11-08,false,98440547.2909,10:51:53.8 +2090254603,40499108228543968,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.910,1955-02-13,true,1309077724.67,22:55:55.8 +-1647304388,46614901571561984,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.910,1965-10-20,false,974644960.769,9:26:7.8 +-623548524,35401079911733168,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.910,1994-04-21,true,70131681.2119,2:5:28.5 +1179045756,43058934957650688,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.910,1998-07-24,true,1145809874.69,13:56:46.15 +166914632,36565791845289616,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.910,2013-04-13,false,1009844395.06,15:4:22.15 +-1988443397,33391456059105444,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.910,1980-08-14,false,85359882.1586,2:27:47.33 +1513112437,1086278312002631,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.911,1990-01-03,true,1180673491.01,17:12:49.1 +157670849,76758452794347024,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.911,2002-03-14,false,564578010.724,2:30:27.12 +1425585597,15094792788601724,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.911,1994-03-13,true,1093835201.02,5:6:8.52 +1967902724,58390060326273688,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.911,1959-04-24,true,1023947069.23,6:7:21.13 +1444203576,47802114866155080,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.911,2008-10-18,false,1243871919.01,16:34:33.18 +-1411454443,90146811942797680,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.911,1995-04-14,true,79193972.2394,12:3:58.17 +-579524487,59987975213918200,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.912,1971-11-22,false,926279464.33,21:34:30.40 +167174723,18619094236703796,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.912,1986-08-21,true,883436451.751,11:31:11.36 +533530058,4682355105911931,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.912,1996-07-03,false,1365229376.05,12:34:11.24 +1976480965,67853674030147576,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.913,1961-11-14,false,1084898006.68,8:4:14.46 +-2135175944,73353024590817760,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.913,1952-09-13,true,581343213.712,19:4:44.58 +17149346,42672678879906952,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.913,1993-03-10,false,1235850236.85,3:41:1.1 +-1582137303,380930230359162,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.913,2012-09-26,false,1018595366.85,22:52:38.45 +372142794,3369929571544166,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.913,1989-03-20,true,1144366111.67,9:14:22.37 +1091434583,20073872817585676,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.913,2002-03-14,false,1135396139.29,14:46:40.49 +-1413587118,47587640667228496,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.913,1985-10-18,false,1109827015.52,11:28:41.8 +-1388909298,31215551857220740,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.914,1987-03-01,true,675084460.588,7:39:10.48 +1773452843,21509001436502108,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.914,1958-05-26,false,785765266.557,10:11:53.54 +621297358,6239312685025628,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.914,1977-07-27,true,574074838.944,11:29:24.37 +-988237995,57858355917960144,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.914,1989-06-27,true,7446641.15169,20:9:21.22 +-1393473802,32805871255509536,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.914,1959-05-11,true,211654991.486,11:10:21.30 +1508872524,54397418519727344,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.914,1959-09-24,false,856117660.818,3:32:21.44 +-966890646,5981487760978104,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.914,1956-11-12,false,539661814.293,10:5:54.13 +1952556002,7881271567945984,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.915,2006-07-24,true,51900564.862,8:47:7.23 +-656408147,64525308730292792,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.915,1969-03-26,true,445121453.172,7:10:4.17 +-97539417,27105201655172292,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.915,2003-11-14,true,1166248097.46,9:35:3.21 +1212210766,23211705026260028,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.915,1964-03-14,true,352202200.886,9:6:34.47 +841941813,87585144637125760,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.915,1989-07-19,false,1322039417.88,22:17:12.5 +-733220658,89018260949013456,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.915,1984-06-08,false,708377791.142,5:1:32.9 +-1349408390,40756825061729016,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.915,1973-10-24,true,502532787.313,12:4:47.57 +-961077893,20657886132134216,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.916,1968-10-24,true,171009338.429,21:31:32.1 +-421057244,19152800962261196,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.916,2006-04-09,false,1413655010.3,3:39:4.17 +2099157999,35664790053717004,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.916,2008-04-15,true,322132119.3,8:44:8.12 +-1125307218,45469295000763616,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.916,1963-03-25,false,428203205.999,21:19:8.7 +-1876891149,67826492208383160,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.916,1990-03-12,false,77233202.0903,15:32:54.32 +1861421718,15949371696736144,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.916,1954-07-17,true,951312535.075,7:33:50.17 +-1326969439,72761439635129536,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.916,1989-02-08,true,1052016559.38,1:16:55.34 +-2090493435,60981847773606536,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.917,1970-09-26,true,1099832321.04,2:9:29.27 +-1086688276,72593709884689696,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.917,1959-09-11,true,778044862.211,1:46:37.14 +588491310,29999143143775252,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.917,1980-03-12,true,66666556.847,7:24:56.48 +-1663539938,80920047905609440,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.917,1953-09-15,true,847833197.426,14:29:21.38 +-768199584,24568619528002252,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.917,1958-05-26,false,166335486.518,22:57:31.56 +-226713079,55771425710720408,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.917,1966-02-17,false,94689245.2661,10:52:16.23 +1991995892,66260365214604432,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.917,2005-01-09,false,93658532.2597,11:2:42.21 +1088076724,71775736708938808,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.918,2007-03-13,true,1228556547.94,6:33:54.2 +-656337266,80973889892480096,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.918,2009-05-04,true,526005632.157,10:29:16.5 +-234048434,271447212388679,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.918,1975-09-03,true,117178098.477,1:38:10.13 +-471862213,9038348457875088,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.918,2004-01-06,true,1334632454.77,4:53:57.4 +1181770944,64323764557401376,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.918,2006-09-22,true,681797268.278,19:20:4.54 +722498048,39440394019841712,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.918,1986-03-20,true,706662213.128,5:43:11.55 +635290907,73386741294175904,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.918,2012-02-18,true,851889107.224,16:54:25.12 +-373417237,30692263555541268,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.918,1963-03-22,true,59832506.1693,5:26:27.41 +1707910306,20889910850954680,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.919,1997-01-24,true,262873027.548,19:35:5.42 +-1258306483,17515704922778328,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.919,1976-05-07,false,775127088.953,5:7:54.24 +685188007,67146247020620096,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.919,1976-08-14,false,518682732.408,15:10:19.2 +-1631214022,62662369859659928,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.919,1996-06-22,false,621709848.317,11:30:44.50 +1216947697,33234537994599096,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.919,1951-02-15,false,11425626.548,22:48:44.23 +1116743292,63451661093903656,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.919,2004-10-13,false,128673820.459,16:44:58.3 +-1759781660,55164608418481032,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.919,1989-06-01,true,533984108.45,16:21:18.58 +-2105458761,63632865890388976,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.920,1998-08-18,false,773412770.301,1:19:6.56 +-1760160154,58440204921162688,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.920,1989-07-06,false,532712219.827,20:54:30.21 +-2011108995,82896119497036336,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.920,1957-02-08,true,849410180.114,10:42:35.56 +864043380,26404298489195520,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.920,1986-04-01,false,124883285.808,14:44:18.24 +91374303,30991952911494996,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.920,2009-01-02,false,976123893.291,4:52:58.21 +86129270,52454872417896944,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.920,1972-05-02,true,189689812.823,19:30:37.10 +-1055800743,90389204170191008,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.920,1978-06-01,false,46120598.276,16:42:50.44 +-171950027,36182545094647920,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.921,1991-01-22,true,1021928814.77,1:58:13.4 +-907608789,8189926444346727,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.921,1975-06-13,false,68609582.6681,17:46:46.40 +539136933,75830878832445440,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.921,1981-08-18,false,1431965391.25,20:58:26.9 +1064686500,237610071123947,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.921,1960-09-08,true,1417019411.1,13:7:36.5 +886604675,24399441434199368,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.921,1975-03-02,false,229008070.008,22:45:11.54 +-2005361978,70873899346726248,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.921,1995-05-01,true,1298034933.0,3:15:47.39 +2101507021,7638716035883715,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.921,1966-05-25,true,1432017616.38,6:8:32.53 +-1572228105,61801774111012128,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.922,1983-01-14,true,1396648421.58,11:29:1.55 +-320396717,10320687535085732,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.922,2001-02-10,true,469790752.712,5:36:25.13 +-473463042,61955678822398488,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.922,1980-06-10,true,1148906493.61,7:27:16.5 +865314511,63482542241653576,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.922,1970-11-10,false,604334252.152,11:25:32.50 +-685279314,17959393258131570,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.922,1968-02-16,true,931585062.879,3:49:58.1 +-1049707505,79573676485186144,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.922,1978-04-22,true,248798481.01,2:13:35.41 +-273352287,31668976939668428,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.922,1969-04-08,false,1402065476.59,6:3:11.14 +-1376593026,16781367708334090,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.923,2005-11-06,true,253563069.575,6:47:53.1 +-1539927274,36618567025781720,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.923,1973-08-20,true,1267520082.61,4:28:31.28 +-2068857947,18956236586981436,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.923,1980-03-17,false,1313257325.07,4:35:50.19 +137217491,25135133427480740,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.923,1983-07-01,false,802891714.817,2:24:3.36 +-1905771304,51448928115547824,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.923,1970-09-22,false,571487478.056,21:13:55.51 +780670557,25062297065588848,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.923,1969-01-17,false,505275394.129,3:14:48.17 +-261700945,78574736910879280,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.923,1988-05-13,true,428107950.745,14:15:16.43 +-175902255,16639470544771830,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.923,1971-08-03,false,356200325.657,7:47:4.49 +-2015789916,55626327510272464,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.924,1954-04-04,true,796592294.346,14:18:4.23 +893388577,54145368335861312,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.924,1967-05-20,false,1232926486.19,19:32:5.18 +186967831,60077634100005128,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.924,1975-08-13,false,137213722.602,11:26:43.49 +-30910513,43328079866489112,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.924,2011-02-04,true,615911683.43,19:15:20.1 +367683978,63786427514393056,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.924,1954-09-03,false,948517241.644,7:10:27.7 +1946607756,15365207468130510,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.924,1977-05-18,true,800287625.824,5:40:26.58 +-1754912968,15904649150937724,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.924,1967-08-20,true,1218121105.74,20:25:27.20 +-3267045,45054564767472264,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.925,2013-07-01,false,77517153.1434,16:54:46.38 +-2113632781,48911328403192200,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.925,2004-03-27,true,443493021.859,3:17:34.21 +254477380,23412199914531236,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.925,1950-11-03,false,635814187.448,19:4:37.15 +-1534061454,86113515923408256,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.925,1961-11-27,true,423292344.362,18:26:10.36 +-686780654,7351016620154624,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.925,1966-02-03,false,430923551.731,22:29:28.13 +-486000744,31410153296196332,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.925,2006-07-06,true,769426386.053,10:31:42.40 +444156981,28037544817401684,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.925,1958-08-03,true,559378926.422,21:58:18.44 +-233024001,84145250056853952,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.926,1967-11-24,true,1317917120.25,4:17:55.48 +-809689612,89937716494237088,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.926,1960-08-09,false,13308052.5918,11:43:23.15 +1965276381,63138719735232224,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.926,1996-03-22,false,903538422.679,12:44:29.7 +-1714025812,37327186710353136,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.926,1970-09-11,false,1410343205.71,22:20:30.29 +-1630971522,54123717642861192,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.926,1976-03-21,true,868968252.161,4:44:43.37 +-1396931536,26023216621360896,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.926,1982-04-08,true,1414882080.12,1:29:6.27 +-1376576916,6859942108168438,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.926,2009-10-01,true,997728586.409,12:32:57.8 +2016429322,19071921535192404,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.927,1965-01-24,false,564014428.241,4:36:42.9 +-221324430,46678765469844472,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.927,2002-02-19,false,182037392.386,22:47:18.31 +-939997410,80145179301678912,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.927,2003-03-25,true,685073801.066,4:52:55.16 +1402984158,13245207978887434,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.927,1965-03-10,false,749286528.382,8:38:2.37 +1299859160,16570643583850896,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.927,1990-02-06,true,627383635.073,5:10:18.5 +-918585657,4928611086204744,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.927,1992-05-24,true,1149969855.52,4:35:13.26 +1294767084,25392559172052604,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.927,2001-04-14,false,1394114729.59,18:9:4.14 +452946386,84486396172691680,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.928,1969-10-25,true,359278335.755,22:40:49.42 +878595057,43279674117223824,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.928,1961-05-17,false,958425350.199,6:44:3.29 +1929190438,62766593234848264,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.928,1972-05-19,true,1422645768.54,18:26:28.35 +-85053992,12167689348728238,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.928,1973-08-11,false,108689052.179,3:37:5.1 +-833469548,4424149789566935,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.928,1952-05-05,true,74351484.7596,19:20:28.48 +1811687557,68418874194189248,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.928,1998-10-11,false,695246445.166,14:9:53.43 +-1864788907,19797819172576852,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.928,1967-03-05,true,50813537.161,5:49:45.13 +-1326744370,43192285927102464,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.929,1978-07-06,true,823965731.699,4:3:40.34 +666868361,47163870024865312,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.929,1962-10-26,false,1424861104.64,22:29:17.58 +348684007,47867931790622192,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.929,1996-03-23,false,1390488844.64,11:49:42.54 +-1714980126,44692488661657048,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.929,1977-04-26,false,263765886.695,19:33:27.53 +-1608293917,88418151553574112,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.929,1965-09-19,true,51690967.5833,13:22:24.53 +-860785245,88061270307024400,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.929,1990-01-15,false,406335935.672,9:23:17.37 +-1577402528,26127439524442820,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.929,1989-01-05,true,1224934812.24,19:31:58.42 +-1744090339,30148380416723036,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.930,1987-06-26,true,556097666.596,1:48:13.43 +-773410453,20432351794648168,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.930,2001-08-26,false,892895687.771,16:28:16.7 +438107903,67462682291245048,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.930,1978-01-14,true,801423804.847,2:23:17.28 +-368633585,750264626849157,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.930,2001-05-26,true,717036666.434,2:20:19.51 +-595161293,7778081245321503,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.930,2005-07-18,true,647554513.397,11:5:29.19 +-658785921,31753265747020052,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.930,2012-02-24,true,895237314.398,18:5:40.37 +-717469827,59746395962475616,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.930,2009-06-26,true,139763944.082,11:11:58.21 +-725201082,25633220046860576,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.931,2011-10-20,true,686238943.669,19:27:41.1 +-343236537,23644793133955452,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.931,1974-08-08,false,1396478659.79,7:53:42.53 +1933663000,38315969264144560,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.931,2007-08-08,false,652307955.47,18:37:48.42 +484430560,89584190593045184,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.931,1974-01-23,false,207111937.747,6:57:39.44 +-1031779658,49850268437397288,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.931,1998-08-02,false,889632128.983,20:5:58.39 +917116011,40415225468154888,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.931,1961-10-25,false,1207781277.66,10:27:11.39 +-312979226,8824913136832369,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.931,1957-06-20,true,531532558.113,3:28:53.4 +524292498,66877278488442448,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.931,2006-04-15,true,157638614.419,12:19:11.48 +-2011057012,42320350923210336,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.932,1981-04-25,true,992543357.729,11:19:36.43 +695300169,35927842118636884,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.932,2011-05-24,true,577675777.213,6:15:37.8 +1144112817,74966028289758064,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.932,2010-03-10,true,1044364517.08,6:23:17.16 +1822956926,86699531639226736,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.932,2000-03-19,false,284424138.971,12:8:34.11 +-1472090481,12949710466678200,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.932,1987-01-07,true,653742067.398,12:9:7.41 +1733359993,30637076084919492,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.932,1961-07-23,false,1081924098.34,1:29:16.39 +-1203221344,29366785399589624,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.932,1975-11-12,false,699709579.285,3:45:13.39 +-1468236040,61503687934177392,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.933,1969-11-17,false,795209572.458,12:58:54.33 +-1902184279,76164555263523312,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.933,2013-11-06,false,1299038736.92,20:23:43.50 +-1076493160,90308308953699600,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.933,1956-03-19,true,172821932.752,20:41:15.32 +-1575908952,89647869761862992,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.933,1978-04-20,true,339917558.126,3:28:33.45 +-795385685,80197704516841312,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.933,1968-02-17,true,1306815905.3,3:15:47.5 +-271656874,76694050495440480,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.933,2007-11-15,true,246803574.574,15:14:37.29 +1624894642,88993221700376208,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.933,1990-10-25,true,1166452370.88,5:14:56.23 +77720678,39828272550838896,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.934,1986-06-06,true,884663663.681,5:28:24.6 +741938467,76234481362002624,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.934,1988-06-23,false,436113719.229,20:39:29.44 +1782329671,5875007846754540,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.934,1992-05-15,false,714588551.539,4:4:54.57 +-1206794847,37810248664535000,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.934,1999-08-20,false,1213049293.49,21:32:43.28 +761785100,73608966661801408,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.934,2002-05-20,true,593329844.413,15:36:36.10 +-1765277560,80174784755491296,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.934,1950-01-22,false,37810142.4943,16:1:56.19 +1791688426,67557176313517016,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.934,1967-03-15,true,26587831.9059,15:6:12.14 +-2014115830,53721774652418848,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.934,1959-02-11,false,257381310.767,21:42:6.49 +-1536923372,1728408783620106,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.935,2004-05-08,true,666711376.583,1:16:35.10 +101105704,33422232676574568,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.935,1950-05-23,true,1362671583.62,21:48:23.20 +-335560827,34835232402262428,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.935,1971-01-05,false,576202474.234,10:27:33.32 +-183226292,88089210138066592,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.935,1978-05-20,false,539955373.024,8:41:47.12 +644943736,2353386098570352,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.935,1993-02-04,false,808684906.691,20:30:4.6 +901949254,90035707984789712,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.935,1968-08-15,false,217750432.644,20:41:13.20 +-605919524,78487199569517952,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.936,1983-05-01,true,330781870.763,15:42:58.25 +-1162624132,30112477711269672,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.936,1993-10-25,true,399152181.183,19:20:26.39 +1591949958,46029795859768560,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.936,1968-02-17,true,1140395375.46,2:37:25.48 +-1258235110,2627090885241702,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.936,2000-04-11,true,895452586.99,6:28:16.18 +347941012,78851978402135040,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.936,2013-03-27,false,1289116151.15,2:54:35.53 +-473143932,64953674407032376,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.936,2000-02-01,false,1406293887.61,4:50:26.48 +1501129585,82059742077965104,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.936,1979-09-14,false,1080137866.91,20:2:30.38 +572964674,43039309633902232,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.937,1980-10-05,false,553253836.231,2:58:55.9 +-962478356,82937753086362496,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.937,1996-03-21,true,1212575568.3,9:35:52.22 +658264399,91760932928207264,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.937,1961-05-01,true,400385077.077,3:18:29.47 +1706397065,78059243417929440,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.937,1975-09-07,true,595154477.23,15:12:44.10 +1958700766,84523329715780768,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.937,1991-03-07,false,549043659.165,16:46:26.30 +-868421632,17361296712868896,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.937,1967-07-24,true,858779478.073,2:11:18.1 +1191788298,52046880950792016,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.938,1954-07-14,true,1336977959.59,9:48:10.42 +1118453324,64032881787704744,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.938,1987-08-22,false,390885518.473,14:26:39.48 +919998802,47976096755685000,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.938,1993-07-22,true,1105038060.88,15:43:6.50 +1609340096,5729749057830421,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.938,1956-08-23,false,71681448.5805,10:29:1.56 +-207405978,76281779940342336,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.939,1973-08-21,true,1165480583.48,9:13:45.4 +603801066,1501354047079649,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.939,1975-07-05,true,14130462.4283,5:37:45.57 +1351920489,81844267744894288,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.939,1978-02-24,false,712066838.449,16:26:4.16 +-533252233,51904302036590216,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.939,1971-01-18,false,1135017676.18,3:52:4.50 +19160375,62468137198775656,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.939,1985-02-11,false,643413600.166,20:5:23.58 +179154700,88985989860990288,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.939,1959-02-16,true,1285921998.94,8:34:2.20 +1376278550,50053780250535392,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.939,1993-07-06,false,263500558.303,8:15:21.30 +-1979431883,66993835550761320,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.939,1970-03-04,true,922535662.556,13:56:38.14 +-451576900,51077360112594128,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.940,1999-10-08,true,55804841.3957,13:11:49.20 +-1660937314,19176811600835020,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.940,2011-05-08,true,726330304.431,22:37:46.28 +-2070513179,86119872189472448,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.940,1971-11-23,true,873221275.919,3:25:26.25 +1023628873,83696345332821456,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.940,1987-03-04,true,378692217.849,20:25:3.35 +1065854474,1954539257113211,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.940,1951-10-08,false,428840717.155,18:38:44.20 +-842798811,71909854672987528,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.940,1959-03-22,true,910075097.213,15:12:42.19 +-1358108662,68672899111249840,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.940,1956-01-15,true,494582335.761,4:21:52.52 +-180305876,47953303786209608,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.941,1992-03-23,false,949053317.335,21:53:11.52 +883666220,29769903703722416,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.941,1971-10-02,true,892514570.772,16:23:12.29 +-1175160045,86012696328553520,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.941,1977-05-27,true,358446249.972,5:34:45.13 +1996797423,85807484781478512,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.941,1985-07-16,false,614648156.707,19:6:42.47 +547280464,76071531935296240,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.941,1985-05-11,false,315827330.37,20:52:56.50 +1676341950,18390851457086268,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.941,2006-06-07,false,361687967.142,12:37:26.55 +-278735134,79056126613031456,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.941,1962-10-06,true,374207981.654,21:34:41.13 +2095728107,69856063368298928,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.942,1984-03-04,true,1075024888.91,6:24:16.9 +388297486,75133324604137376,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.942,2004-08-21,false,620185646.51,20:36:26.12 +1774857282,90757986384107216,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.942,1996-06-19,false,691947949.577,9:15:1.28 +-336435657,61983683761611200,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.942,1953-03-23,true,1428742357.29,2:20:39.41 +-520972114,23546746080426436,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.942,1972-01-01,true,541475864.963,12:17:40.28 +454707426,49590023180907328,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.942,1955-01-09,true,576012225.018,12:39:35.24 +1783922383,25627163118008636,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.942,1959-01-21,true,290831349.229,6:15:53.54 +-658391576,34809384063300928,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.942,1963-10-22,false,1155176146.35,15:54:39.45 +353560636,40265023958909448,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.943,1994-08-03,false,470551318.268,2:57:54.43 +504380699,84039637275922448,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.943,2001-01-23,false,1377869133.53,5:25:52.17 +1952772690,3333334216792801,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.943,1972-07-11,false,26706948.572,19:32:26.57 +1563846140,29381872416907860,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.943,1996-03-11,false,474772543.967,1:29:43.46 +924562418,12459185750619710,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.943,2001-09-13,false,507343464.203,8:18:15.48 +-28616286,53534602422722824,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.943,1960-05-06,false,929076852.682,18:23:40.17 +1203188658,48160217419698296,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.943,2012-06-12,false,276311638.132,13:23:22.39 +-185962478,1795339378869985,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.944,1997-05-13,false,392621290.432,22:22:11.29 +-1175376587,76983811660219488,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.944,2006-11-02,true,59169963.3594,2:37:24.49 +-2021993098,35348057150203536,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.944,2007-03-02,true,1219917719.3,6:12:53.21 +-467525836,6302722895664773,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.944,1974-02-15,true,154984018.817,9:12:16.31 +-1235612909,50216081227879240,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.944,1993-06-23,false,437023181.026,11:15:38.43 +1577002919,36969463731088432,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.944,2009-08-15,true,67569081.7558,5:14:42.10 +2118807690,54665899826511528,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.944,2004-01-26,true,1312052906.15,20:11:32.29 +-1499257063,481448619836631,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.945,1980-11-08,true,469303397.884,18:44:54.45 +1749546865,71701077791687488,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.945,2000-11-23,true,214401887.927,9:55:17.45 +-1286770758,73744559049439104,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.945,1975-10-12,true,139129338.495,8:43:18.29 +158533429,24214341281110232,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.945,1994-05-23,true,679989241.899,5:46:27.15 +-1950848256,33451372395944040,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.945,1975-05-25,true,1263945029.46,17:1:41.51 +1457459918,29261126350918392,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.945,1962-02-24,true,937582059.621,11:40:54.37 +2094291798,53044266052210896,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.946,1969-11-23,true,691910701.314,1:36:48.33 +-464936338,15495521083665346,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.946,1968-07-11,true,285376943.748,1:12:15.22 +-47460254,18657735897609852,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.946,1973-04-15,false,851575683.509,7:9:8.48 +32924882,79490845390362336,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.946,1965-10-27,true,484094099.624,15:44:34.4 +12046002,65868373328420680,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.946,1977-02-23,true,1259554626.66,7:39:26.20 +1026041839,63095804181443608,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.946,1964-03-27,true,415273837.278,19:38:23.15 +2067632281,88293690153009712,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.947,1976-08-15,false,1274119336.69,18:14:20.15 +-2006865657,41333419024656344,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.947,2006-07-07,true,1252638113.27,6:3:38.2 +1275168895,1328931008470149,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.947,1986-03-23,true,284581075.239,6:24:46.34 +-668709567,82630453073655376,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.947,1990-11-09,true,1352687453.43,20:19:15.52 +-1210032162,41099287855460888,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.947,1957-01-10,false,1298126939.35,22:53:31.20 +605294243,4315897546492569,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.947,2002-06-17,true,1180411367.9,20:29:58.32 +-1934734826,109482433895270,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.948,2008-11-16,true,154804500.468,6:5:7.37 +-1342069188,10184531295877662,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.948,1988-11-17,true,599600417.244,22:58:6.24 +87652235,89049168022543840,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.948,1963-05-10,false,268310068.998,20:42:27.5 +-659455563,70350434091004488,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.948,1989-09-01,false,540205825.854,19:40:5.55 +-467140689,17896315375039816,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.948,1985-02-21,true,1270912084.5,4:37:14.47 +-36723436,13881780208098806,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.948,1955-07-07,true,1081132846.52,19:54:41.27 +188206252,48329124911735080,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.948,2002-09-12,false,711132079.962,15:51:42.24 +-1478687495,23504361069016168,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.949,1971-06-04,false,246910830.76,20:7:5.43 +883827250,49824607364490672,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.949,1975-08-12,true,539781876.191,15:30:48.44 +417998004,60624438064620040,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.949,1966-01-12,true,1090322378.0,18:26:18.42 +1970605850,37909662250057808,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.949,1973-08-12,false,218313852.246,10:55:16.27 +-1048522590,37107084259498712,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.949,1958-07-07,false,84636226.858,1:8:5.13 +-2085500293,52701107777243304,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.949,1998-02-12,true,582127205.191,7:42:55.46 +190951451,55440903992556968,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.949,2000-07-11,false,355976036.502,11:10:12.7 +-1604354185,59704620658278584,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.950,1963-08-16,false,1077212775.6,12:58:43.41 +-1602057933,63357029499441688,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.950,2008-07-08,true,1161901707.6,7:55:55.14 +-1069145315,22854265561263188,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.950,2005-06-02,true,199066440.625,11:44:34.41 +-1302557537,78206196535241920,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.950,1997-02-05,true,1204955014.24,8:18:52.15 +-498178186,42027829977291304,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.950,1993-07-18,false,926567597.471,17:37:31.9 +76034740,15749539483397602,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.950,1985-08-11,false,197543033.715,5:41:41.16 +-1029190693,36930636771681568,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.950,2000-01-01,true,1408258732.7,21:37:14.54 +-7114083,21891056570174456,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.951,1952-10-23,true,159639614.491,13:52:42.7 +-1331068085,49490020432707992,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.951,1968-02-26,true,884369522.934,13:41:35.40 +-1574217225,85738640351476976,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.951,1979-06-07,true,903643131.47,12:51:15.42 +-179223719,56109206273264464,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.951,1982-07-18,true,646888169.563,16:29:27.49 +-340994262,32298468453471244,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.951,2009-06-02,true,453326526.376,5:26:14.55 +-195610522,19487467982770492,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.951,1984-07-03,true,427090589.969,13:25:13.58 +902656721,7268702543107973,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.951,1971-01-16,true,1366222264.8,21:1:38.42 +1309288464,61679824729211472,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.952,1969-04-09,true,454531396.989,5:49:43.9 +-552197329,54291685014408048,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.952,1960-04-17,false,438595464.773,10:41:52.36 +2001106696,73998457119370608,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.952,1955-08-13,false,742822613.596,20:58:42.20 +1382155080,76672624728165152,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.952,1973-02-11,true,840321214.291,13:17:20.9 +1929636501,74100212649142320,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.952,1966-04-01,true,1238625289.11,2:9:24.51 +1404750791,5094409164727112,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.953,1962-07-20,true,858253699.526,1:55:15.21 +839274860,7622103432287785,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.953,1962-03-23,false,428929084.464,11:32:51.30 +-1180242111,37803807397818080,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.953,1970-02-01,false,808741532.295,21:44:18.28 +1156768209,85053749210293856,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.953,1976-10-18,false,1341035273.26,6:19:47.40 +-1061687636,56718670224546688,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.953,1972-10-26,true,234142391.809,4:5:35.14 +599518095,18251753836607436,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.953,1960-11-20,true,852587523.0,3:37:36.22 +-716021834,12233531715134270,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.954,2004-09-25,true,974781022.165,21:50:34.38 +1975913547,60793472909792768,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.954,2006-02-04,true,197145752.319,4:19:40.8 +-1635464605,39202510642534464,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.954,2006-04-25,false,1115762355.87,16:35:52.44 +1910784035,28779611889771040,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.954,1980-03-24,true,1000623139.92,13:57:32.15 +697648460,29339236101263444,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.955,1971-07-20,true,1419906069.32,1:7:26.45 +1556707214,29845020101950496,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.955,2011-02-12,true,520533877.958,21:56:29.53 +853339216,42863073350701600,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.955,1977-05-24,true,1093454786.98,20:53:22.28 +433344315,45146301849692896,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.956,1968-01-10,true,719911388.116,11:26:55.6 +-2129464752,65568044075127848,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.956,1970-09-12,true,938664994.174,1:25:52.4 +-801799176,82208164297365264,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.956,1958-01-03,false,1074407128.04,4:55:52.29 +1780393215,4549759875820411,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.956,1991-10-23,false,280459629.566,21:34:54.39 +-576294926,12197656427859742,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.957,1996-11-09,true,304192074.962,19:57:39.16 +1898679942,30931982097278412,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.957,2004-04-18,true,1151791183.94,13:49:37.26 +-1498928215,45941763243013960,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.957,1951-01-12,true,1020651118.29,10:6:46.39 +1061124944,44137456495638808,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.957,1965-06-21,true,769022797.907,18:47:55.58 +2024183740,62007781900226920,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.957,2003-01-11,true,601340127.349,12:47:15.4 +1804360390,3271950828543580,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.957,1990-04-23,false,1222606760.35,12:44:8.41 +-57697703,81241540102651552,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.958,2001-03-11,true,280627996.669,11:38:22.56 +601440826,42875897732467416,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.958,1952-06-19,true,118686907.878,16:34:31.56 +1748200775,67418525171267608,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.958,1966-07-12,true,668140563.074,9:37:52.20 +1113288855,1789912293393797,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.958,2010-01-19,false,19708375.8449,3:28:21.32 +-249546774,11734348117711350,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.958,1980-03-05,true,474079831.907,2:57:53.49 +-1886622606,51561532944693024,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.958,1963-06-22,true,1136355093.69,1:36:16.7 +-209070493,57051979054964352,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.959,1988-07-27,true,53942015.692,10:36:35.30 +-1001919734,21393457783697408,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.959,2011-08-01,true,45450033.3947,13:54:45.14 +-212135459,23623920029429280,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.959,1998-02-18,true,965176570.269,22:40:18.1 +1115391987,52984475807050392,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.959,1975-07-15,false,369957609.633,11:51:53.20 +-1240137060,89043881304429216,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.959,1957-03-22,true,83951652.2976,9:46:6.16 +1685965771,52112258774993672,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.959,1961-08-26,true,995185544.606,21:13:14.11 +55979997,25431066748304376,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.959,1966-02-19,false,1278939234.68,6:39:21.20 +1959684866,83443772318865904,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.960,1983-10-13,false,775090724.316,3:6:47.34 +1948376380,12562004930217800,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.960,2013-03-18,false,481067589.482,13:19:40.15 +-1013238796,13471995655464540,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.960,1990-06-15,false,248353384.415,16:23:15.40 +-78022985,68563844664598016,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.960,2013-02-07,true,1360727440.02,1:8:4.33 +-988316635,67519574772366904,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.960,1988-11-25,true,1058829798.76,8:21:41.39 +432047957,89014632196050992,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.960,1978-01-01,true,782342622.71,15:20:1.28 +-1900493893,8195135356365742,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.960,1985-03-17,false,1161785665.56,6:42:24.7 +1791477730,22469849976298312,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.960,1967-09-03,true,432043604.834,7:4:41.47 +-247723606,21614980553709772,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.961,2010-05-15,false,969077484.097,22:21:25.5 +2128878542,67876852844822448,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.961,1983-06-08,true,372960589.195,14:30:44.19 +1775664925,8057523332040510,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.961,1998-07-26,false,213553941.819,7:1:35.55 +518329368,40956465056398064,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.961,1973-05-08,false,1201190684.9,19:15:21.28 +807108240,40340433845238720,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.961,1971-07-10,true,61954054.4456,18:57:27.13 +-1607983785,43927216612962264,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.961,1996-07-15,false,1416675898.68,21:12:30.9 +939497575,16389364363683288,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.961,1999-04-03,false,1353286890.77,21:42:39.14 +-13724260,9092163701784392,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.961,2000-08-04,true,1171465400.81,12:18:52.20 +530319712,34680212259190804,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.962,2011-10-19,true,1091254596.12,4:41:58.25 +1358177373,3843316874232729,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.962,1991-08-24,true,647968702.218,10:12:16.11 +2056592532,81860997411129120,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.962,1998-03-13,true,925913594.414,5:2:4.48 +-1344302912,85288967887437184,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.962,1975-07-19,true,752815111.04,4:24:33.54 +-761203176,19479142305334376,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.962,1987-11-27,false,822097035.009,6:5:14.7 +-1167767397,51859183729563096,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.962,1955-02-19,false,394767393.614,7:33:5.43 +1752082198,3959707282426747,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.962,2007-09-05,false,1094015614.86,10:22:12.15 +2552761,50006195341654920,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.962,1994-03-12,false,41112712.4879,9:47:54.18 +-2037624587,25531246947716372,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.963,1951-07-03,false,731713570.617,13:20:58.55 +-478098619,58225304342669480,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.963,1993-01-04,true,251276820.546,9:42:30.36 +661515229,14143539101104394,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.963,2012-02-25,true,322576249.763,21:43:45.8 +1125178996,38221545612013176,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.963,1972-11-03,true,1336285496.22,20:56:28.22 +-198206683,25168083674509640,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.963,1968-01-06,true,600325080.355,7:15:35.56 +1557248305,10687229759821712,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.963,1972-10-24,false,198774612.157,3:43:10.23 +1034793216,17412867542498294,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.963,1996-01-13,true,416768627.411,7:41:10.34 +-838803662,7994639751962266,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.963,2001-06-14,false,896219543.163,1:34:46.56 +894836202,69397590997564640,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.964,1980-03-13,false,18856060.8602,13:45:33.45 +-1576447992,63336780958231264,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.964,1960-11-27,false,964220599.562,22:51:37.7 +-782187477,76157023210364144,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.964,1984-10-12,true,175745360.456,20:43:5.33 +1756707290,18994612948358144,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.964,1993-09-08,false,992158925.072,20:40:14.17 +707708472,73762227201942864,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.964,2000-08-06,true,974118970.285,8:50:53.24 +-9147396,20055333084280168,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.964,1987-03-09,true,353537342.257,8:10:10.36 +-1057617923,69134023556365536,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.964,2003-10-10,true,1213240887.3,9:35:29.39 +-313965265,44616867336031600,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.965,1968-02-25,false,1212183767.32,20:36:17.24 +-1906963279,43377242076325512,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.965,1988-07-23,true,534276118.723,16:48:34.15 +1827188485,65010009360568632,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.965,1986-10-19,false,562948599.757,4:15:40.54 +1233764475,77253528573332672,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.965,1962-06-18,false,441027111.001,1:7:58.43 +-519509853,39425886843278760,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.965,1963-10-06,true,747928167.512,15:12:31.3 +1333431960,8510389604341525,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.965,2009-06-21,true,38443444.6824,17:9:47.20 +-421859873,46344392806290600,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.965,1964-06-26,true,343904875.782,6:10:47.52 +-964769327,62008461055205880,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.966,1978-03-07,true,168043366.461,19:58:12.38 +1194993908,60191333159889328,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.966,2009-11-27,false,652554741.791,15:36:8.34 +1310303229,90709986910737936,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.966,2006-01-09,true,558696772.076,1:23:50.4 +-697945684,36573147521622296,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.966,2009-08-26,true,1114485252.86,12:32:10.23 +-1180213990,66201016328794808,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.966,1969-11-13,true,528075170.906,6:9:29.24 +1543147911,56711027371047784,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.966,1953-07-25,true,216475157.619,16:43:58.41 +-1718034904,65711136396178112,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.966,1990-01-15,false,1209914827.7,15:49:26.52 +-1508706719,31050784573221684,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.967,1961-08-16,true,225751256.016,16:39:49.54 +-1158029264,75622999853959360,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.967,2004-06-06,true,991942833.875,8:34:44.48 +-1873487038,51366114674649880,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.967,1967-08-22,false,880281723.404,10:19:34.29 +224765846,32829752950858956,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.967,2010-03-14,false,163637823.015,14:34:30.57 +887012957,3302410240208486,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.967,1963-08-02,false,1177655525.58,8:13:35.30 +-886189757,24193440989036092,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.967,2012-01-05,false,117288816.512,3:25:48.54 +-1994656140,43583270654079800,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.968,1972-07-05,true,468846842.669,17:7:9.27 +1859216683,56698204374425960,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.968,1991-06-27,true,1304525874.01,4:19:28.29 +-1494390497,48714608662803264,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.968,1992-07-16,true,942067383.263,9:39:26.28 +706815110,79521894284994272,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.968,1980-11-25,true,101887738.803,1:33:51.8 +1805744680,75155127277399936,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.968,1982-11-18,true,1022594742.74,22:13:20.8 +1313674113,25940410597554104,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.968,2002-01-27,false,1116628814.48,17:26:58.33 +1878470217,70224763536596024,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.968,1955-03-08,true,108738986.791,11:7:38.51 +-630179614,7764769258776904,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.968,2005-06-23,true,927190943.989,18:30:28.51 +1160121704,23815404550575352,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.969,2001-06-23,true,881840820.009,10:35:26.11 +-1700238057,17061529150718228,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.969,1982-09-07,false,1128620614.84,21:43:4.52 +-561755450,64287424371605096,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.969,1994-10-03,false,1222674079.21,1:47:16.19 +750601274,42459486202537336,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.969,1972-07-15,false,650294557.307,10:19:32.37 +2057149562,75294469019348608,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.969,1983-04-03,false,756679507.693,1:54:56.3 +1629500924,57839992937036400,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.969,1971-11-14,true,114003070.877,3:30:5.20 +-1363926182,46058809439650784,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.969,2011-06-14,true,359931268.087,4:22:56.24 +-219995115,30830641383144092,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.970,2008-11-18,true,576624877.951,5:10:38.37 +1866041547,77375871710695584,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.970,1971-07-19,false,1354312317.02,8:44:3.23 +-342671400,48759297710640672,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.970,1994-10-11,false,617966241.049,12:12:36.27 +-816789283,82657790101116048,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.970,1975-07-11,false,85425443.356,3:54:14.26 +-693253642,63238169292277768,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.970,1970-11-21,true,411104969.843,22:15:13.28 +123911191,29159158784897404,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.970,1959-11-07,true,746312181.708,8:34:16.29 +-1454849950,61381805022038776,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.971,1955-08-24,true,1036061826.61,12:24:14.44 +2042787904,54641695360801384,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.971,1995-08-13,true,895664270.943,2:14:13.33 +509856016,78761198952458512,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.971,1985-11-22,true,156442023.839,21:28:20.48 +1256913168,49001201147460208,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.971,1963-02-17,true,441486909.401,16:28:25.16 +-151037476,73311840387576224,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.971,1974-05-08,false,230703058.517,15:23:16.48 +822216300,82709027994765360,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.972,2013-06-20,false,1389622132.26,8:46:37.26 +551831373,72059455038409968,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.972,1964-09-15,true,319425726.229,16:39:41.42 +-638784036,78456071792207888,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.972,1967-03-03,false,1392487822.85,5:17:46.7 +-1523032427,17364530517317848,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.972,2012-02-13,false,1241232048.81,6:3:45.23 +345144055,13353922562902896,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.972,1982-03-16,true,983768711.432,5:2:3.27 +1105681278,21573338742321236,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.972,1960-11-11,false,304626456.788,11:58:44.29 +135444849,91453823596979248,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.973,1971-02-06,false,840167821.152,15:42:52.23 +-371936298,53921453284129448,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.973,1957-04-04,false,933255392.169,7:53:19.58 +1133854035,47129307640487096,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.973,1994-09-24,false,371494850.802,21:11:2.5 +1784411331,19215342776003952,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.973,2003-02-17,false,930001475.023,6:12:17.49 +47144251,59036598764268720,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.973,1993-03-01,true,505309279.619,19:30:38.15 +2131884996,83668953960940880,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.973,1970-06-12,false,95497781.7848,14:1:15.43 +-1316230996,40562940463045656,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.973,1970-08-03,false,850753728.122,20:18:47.13 +987680448,58823816464755608,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.973,1972-01-03,false,928362030.663,5:32:30.14 +-1468525795,47081626530358752,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.974,1959-11-05,false,141359540.057,13:16:13.42 +-1394054139,23614217626865932,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.974,1990-02-25,true,1168818767.95,18:34:26.45 +912145214,87666874671309072,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.974,1970-02-13,false,1419339010.54,8:51:19.51 +-138077686,73856194471193888,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.974,1996-01-04,false,163839170.585,1:54:34.13 +-595033752,24672651553523468,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.974,1959-11-18,true,960877918.959,3:48:46.44 +1818209616,44883448590654200,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.974,1980-06-01,true,1134343044.14,10:17:35.29 +-1455053632,63999627600026256,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.974,1969-10-25,false,577018603.597,5:2:2.51 +430102560,34794415041014212,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.975,1966-02-21,true,266863563.475,14:39:22.9 +539155380,55077486810533640,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.975,1956-04-11,false,231760548.404,18:17:7.25 +-500275197,46096621069370720,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.975,1996-09-08,true,201641876.295,7:15:57.43 +-231618040,10627312159781038,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.975,2005-08-03,true,780530718.827,4:58:32.47 +-28149183,75056239513728704,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.975,1979-03-15,true,562932390.163,7:54:18.58 +805441594,25456822557194252,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.975,2008-09-25,false,613207637.764,15:13:13.38 +-669851031,56020535747624888,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.975,2009-08-08,true,442126381.348,20:51:36.38 +454971775,70499155537269240,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.975,1991-02-25,true,1105298832.94,15:6:58.32 +1111009765,79492337662907072,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.976,1951-02-08,true,351729314.335,1:52:11.5 +-103595805,35818208352048312,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.976,1978-10-26,false,796683937.176,21:54:39.29 +-992225793,5260523340370760,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.976,1977-11-18,true,1054588071.81,10:16:14.45 +435235882,65327935270257208,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.976,1961-11-25,true,584981961.887,3:6:22.24 +-21518051,62155593948620576,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.976,1961-11-21,false,290052222.848,12:42:3.42 +-994207526,55357085094608416,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.976,2002-07-21,false,505727915.283,7:34:23.28 +736945337,18234211382641448,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.976,2000-03-12,false,437190219.5,4:5:5.39 +-923537786,32240315547810744,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.977,1998-10-03,true,447811656.581,16:6:55.3 +-226990450,53566531315447944,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.977,1973-06-15,false,1418122360.69,11:31:4.10 +1079728475,88886247906573488,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.977,1958-05-17,true,1084938763.29,6:5:10.28 +-1068321068,52442614417143816,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.977,1969-04-12,true,768592797.324,15:51:58.15 +-1507424692,56380005175908064,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.977,2006-04-04,true,367174270.313,5:21:57.6 +-1775382379,69245669167637576,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.977,1986-10-10,false,128045087.064,17:9:7.41 +-566059376,66147982804158792,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.977,1951-05-21,true,1088996154.28,16:4:15.39 +-720189527,42204464813844304,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.978,1976-08-24,false,444502254.733,19:14:28.39 +-1688789759,30092627874535588,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.978,1993-05-21,true,730373441.597,11:36:18.44 +736246441,40857910229450224,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.978,1954-10-14,true,329717188.16,17:49:5.55 +-223690892,10140924919884862,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.978,1977-05-24,true,889606967.965,7:1:2.27 +1318841877,88224279290580640,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.978,1953-07-05,true,476123552.575,12:41:54.24 +-307262837,44436398723973664,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.978,1993-04-19,false,221123004.692,13:34:58.14 +-841943474,34896544570962124,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.978,1988-01-18,true,954652328.055,12:18:21.9 +1724385957,77389850562018256,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.979,1960-07-07,true,888790492.381,19:25:33.41 +-946463993,26583801837681912,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.979,2002-09-21,false,324700972.52,4:23:42.47 +1030061116,49958713937749112,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.979,1960-01-24,true,355585401.797,2:10:36.56 +-1904617900,7983033721795809,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.979,1983-05-06,false,49565574.4396,16:14:49.30 +-134395048,9311002459128382,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.979,1989-03-05,false,1139385183.92,7:3:26.38 +-1924299791,18194851900949668,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.979,2004-04-09,false,313378258.913,11:52:34.31 +-1640761128,77848071953804432,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.979,1984-09-24,false,739465137.896,17:37:14.48 +-1790414168,82326044838758080,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.980,1991-03-12,false,706509663.317,18:57:39.36 +-1793075336,23625335630267556,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.980,1989-06-10,true,520433964.04,13:55:40.25 +-684368683,57742898385980472,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.980,1978-02-16,false,64553386.0128,15:54:35.29 +-458486823,71274137291258096,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.980,1983-06-17,false,828607664.936,8:17:29.22 +-2018579133,88780743398883632,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.980,1988-09-03,false,1253778268.69,16:56:39.13 +-1362031949,58426642181090968,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.980,1963-06-13,false,185988164.308,20:54:28.16 +-359415716,33711398244552060,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.980,1962-10-03,true,1263522190.9,21:25:9.48 +-296224306,38133332352643032,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.981,1997-03-10,false,1252820331.39,22:34:19.24 +1581700292,41984779264199872,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.981,1971-05-05,true,1287578665.49,3:32:6.37 +1070111461,18829488777492592,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.981,1961-05-14,false,1019117251.75,5:23:6.43 +-2070045251,71568816068088456,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.981,1970-06-08,true,1409211968.4,7:2:34.37 +1608819692,41143968556043784,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.981,1961-08-16,true,554343019.33,13:17:39.8 +789847835,39280815308740304,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.981,1963-11-14,false,4657447.1145,12:38:40.8 +1095029916,81790382275748208,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.981,1987-01-17,true,200495627.984,10:29:51.29 +-215824986,24915700736574884,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.981,1997-06-18,false,65149655.1094,1:45:6.30 +1741636644,76969616097228864,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.982,1959-01-06,false,991177071.034,9:44:23.7 +-1316103086,49261578969671888,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.982,2010-01-22,false,614316114.327,13:28:41.2 +-259767607,6485769004203326,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.982,1978-04-23,true,1064039089.54,2:21:24.34 +-1129388218,2772796711735347,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.982,2005-07-23,true,1279466418.89,6:1:9.30 +516104424,47293594602588600,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.982,1997-05-26,true,728108278.758,20:41:44.30 +707064282,69211210366556480,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.982,1986-01-16,true,107145364.349,12:38:53.33 +1632774932,63948250288690792,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.982,1956-04-09,true,480195931.097,20:18:34.58 +237781069,50306572974731320,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.983,1961-05-12,true,284356397.982,12:25:35.8 +-918296367,29546169009072088,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.983,2001-02-02,true,234338431.455,2:6:40.11 +-1976198839,65562839131553592,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.983,1974-02-06,true,183648449.705,6:8:13.40 +-709890063,16288957924911810,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.983,2008-03-16,false,1044692083.99,8:25:28.46 +1104655027,58522340211270808,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.983,1989-08-25,false,1268725260.39,3:13:45.10 +1842724036,89534039984064304,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.983,2012-06-05,true,1016420366.0,11:3:3.16 +78445154,77943042990699568,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.983,1984-08-15,true,1268918454.19,17:20:52.23 +-1535956172,79037974028810752,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.983,1952-08-04,false,1362676277.3,22:57:32.29 +-74305969,18732940720548548,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.984,2000-03-07,false,1432031297.66,11:14:20.8 +2089652320,87176123931821664,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.984,1954-10-21,true,1302628556.34,11:43:39.9 +-328595545,30984768114458768,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.984,1965-01-04,false,144565337.094,8:33:37.50 +-1444150335,29001722052406476,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.984,1961-09-25,true,351597213.105,4:55:23.37 +-1073767496,53616726265626808,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.984,1985-03-26,true,539458559.42,14:6:40.40 +467858241,28531911765676280,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.984,1950-02-24,true,870967972.624,13:49:5.17 +-72245888,42177175591239192,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.984,2000-09-16,false,682145768.731,11:25:53.17 +-881578565,48790466183091552,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.985,1953-05-05,false,236616380.812,4:15:7.8 +1741903752,48616586752015480,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.985,2004-03-12,true,1129843569.81,17:46:47.18 +-1036901134,52474039558034432,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.985,2002-05-03,false,953421335.899,16:15:31.21 +347077900,67592713847662288,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.985,1976-02-21,true,1147070555.43,6:45:20.9 +334898966,66674044987592512,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.985,1965-05-27,true,602083851.089,9:53:32.52 +-1586236287,90697468690021792,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.985,1986-06-20,false,465738496.768,10:12:33.57 +1900674451,5038235747536804,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.985,1965-10-23,false,1366669243.59,9:35:58.25 +-1769443837,35469913014527744,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.986,1993-09-27,false,561982214.205,3:20:6.45 +2059064823,64106945484053304,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.986,1995-08-26,true,601599343.084,13:52:23.24 +1491332249,72623830127318672,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.986,1990-08-08,true,779950221.453,4:2:49.29 +2118989538,35973306125081180,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.986,1991-03-01,true,806967815.643,2:2:1.39 +-491544277,64713273929876120,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.986,1959-04-13,true,1202174714.32,2:44:15.4 +148326115,12426236366424014,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.986,2005-09-19,false,845059144.898,14:26:6.29 +491604254,48731370692498768,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.987,2008-06-11,true,77298667.8094,21:38:6.24 +941318981,56809800546533896,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.987,1999-03-07,false,1219261405.67,21:44:29.16 +-977821095,56514527763532296,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.987,1972-05-11,false,1422708867.82,2:7:41.21 +498208789,83796299830854000,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.987,1972-08-13,true,1205557479.4,19:20:41.15 +-1136356091,43605731962834312,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.987,1984-05-20,false,1100255127.4,5:24:11.39 +-1151688992,8821264823047987,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.987,2008-06-18,true,989840306.502,4:5:31.5 +-1058470445,74811596257931600,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.987,1991-09-07,true,288553818.584,22:30:9.56 +577906261,23208913399608248,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.988,1983-02-21,false,80358525.5349,20:14:35.26 +-1908214927,84261339024624768,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.988,1963-08-03,true,1341263145.02,12:33:57.24 +-974249996,85426862419803408,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.988,1961-11-20,true,841450525.855,20:35:22.1 +1503197578,5527220294232474,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.988,2011-05-05,true,1252273986.39,5:53:32.22 +1118293254,86203813761328336,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.988,1988-10-06,false,823066996.847,15:52:38.30 +-1003272256,76628467524442080,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.988,1975-09-17,false,335243926.726,5:40:21.47 +-1386282654,11351593735869768,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.988,1988-09-21,false,673296909.03,20:24:44.12 +1446258847,50989961071169600,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.989,1988-10-22,true,781761262.08,1:9:5.57 +2031056418,1748131856530104,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.989,1962-08-01,false,1307999531.33,20:55:43.50 +-468730948,89962676245274656,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.989,1958-10-22,true,1151359733.77,20:11:3.21 +337772613,44941073326220568,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.989,1960-09-03,true,1332114155.88,16:10:45.4 +1356287914,12176039333837036,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.989,1992-10-21,false,1021559607.33,18:51:17.16 +159229260,74843206515708432,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.989,1985-11-02,true,173765497.615,2:36:48.14 +2108696403,82955134510775264,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.989,1961-09-10,true,36046997.448,8:58:48.39 +1353370037,58075429512768136,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.989,1994-05-17,false,413699116.032,6:3:39.4 +274877763,56014745263102904,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.990,1991-10-12,false,182103312.019,17:14:33.52 +-1804721783,89247605059471168,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.990,1961-08-25,false,1086777947.1,2:12:41.56 +1705836679,65583449501911800,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.990,1963-04-25,true,977517057.109,13:39:29.18 +991197726,77274345613339376,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.990,1953-11-22,false,1140831152.35,19:45:34.8 +-251253957,58398970914534752,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.990,2010-06-04,true,986936304.191,21:52:27.3 +-1325958471,5820487324653793,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.990,1985-10-15,false,1326349914.07,9:7:35.3 +-688259424,61740680194174496,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.990,1981-04-13,true,613066880.723,19:14:37.2 +-1877954031,23676090573487392,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.991,1981-05-24,false,1189218477.54,7:6:2.43 +1378590866,3340237466147778,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.991,1972-09-08,false,1232090444.61,20:11:51.47 +-98382997,49774715336208136,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.991,1986-08-19,true,995428471.159,2:58:9.20 +1366108043,84873326185182320,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.991,1991-07-16,false,22251743.9661,7:53:14.40 +676561823,6107638554965883,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.991,1952-02-11,true,638560576.595,19:36:33.18 +2093601464,10489417232168888,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.991,1982-01-08,true,1162896728.72,9:36:8.58 +1304319945,89931285359261024,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.992,2009-04-23,true,1411884465.13,4:47:23.31 +823957264,43544861091543848,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.992,1990-03-04,true,804967546.318,7:55:18.51 +-421055462,34641117879821836,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.992,2005-10-02,false,1300433820.99,10:36:36.6 +-665783860,66045247643715136,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.992,1980-01-01,true,1196833792.28,22:57:45.58 +-2019836031,71525203459554312,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.992,1993-05-25,true,1237675177.38,22:21:1.29 +-600254633,40538380443489728,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.992,1983-09-07,false,251683898.432,1:36:8.52 +1023022402,2373203656494448,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.993,2011-04-20,false,399163414.942,22:4:40.10 +-726764529,75587796436537760,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.993,1990-05-08,false,965526128.833,17:48:40.8 +-490094936,71647858425938640,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.993,2000-10-25,false,1117710952.24,3:12:21.19 +-1762205085,42845026330295488,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.993,1995-04-20,false,453449919.034,16:17:48.16 +-1840933785,61793120432381776,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.994,1985-02-11,false,894940756.685,21:39:28.24 +-1674877715,85150900880326832,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.994,1987-09-05,false,130702546.689,2:19:41.32 +-63132180,62554284923556104,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.994,1965-07-17,true,1409276688.06,3:43:48.24 +809380380,32634659590500416,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.994,1985-03-12,true,613765101.069,3:17:51.24 +-1844672647,7415655322782116,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.994,2002-04-27,true,567790102.777,21:55:49.18 +981412592,87251667009923744,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.994,1968-05-11,true,540883765.355,13:37:39.2 +1643531493,39652777862717432,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.994,1991-10-09,false,618623496.558,16:8:20.49 +-452643993,55011815208230352,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.994,1962-09-17,false,828303713.23,8:12:26.8 +-1433777029,9563474637820376,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.995,1991-04-08,true,912561664.271,9:36:58.12 +2088681279,40294892230189888,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.995,1954-02-05,true,640246455.953,7:27:45.1 +490503655,75482194112628352,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.995,1991-01-06,false,161898915.643,8:45:28.19 +760541970,15820891426311106,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.995,1993-10-13,false,1346640901.13,4:43:14.11 +1786469895,57051156361117008,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.995,1982-03-17,true,643154511.242,22:37:18.11 +-1808368438,71626674209573032,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.995,1985-05-23,true,1255540995.49,9:7:20.9 +935572119,80915538068962624,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.995,1969-04-24,false,273329388.164,11:50:25.57 +-532328969,51921131986336832,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.995,1976-09-24,true,1331753343.74,6:29:14.19 +820139255,53762201223775952,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.996,2011-08-04,false,1418782979.94,14:27:6.44 +1299832602,23769657229237504,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.996,2004-08-09,true,374953648.599,13:21:16.7 +-1838625164,84398347866823760,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.996,2006-11-24,false,313299185.332,13:43:48.14 +-1022411046,25838382095150880,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.996,1977-05-15,false,47884738.3952,1:55:22.46 +-848121237,21377536019657136,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.996,1953-05-03,false,1155896320.26,3:16:47.45 +496066078,27380558637812800,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.996,1980-09-05,false,732760985.516,8:10:43.12 +787511108,33702951824242628,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.996,2003-04-08,false,636984743.22,16:31:37.35 +-1551993814,19368727521441004,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.996,1989-08-19,true,274395526.571,14:4:35.35 +-2141525510,80012831284012304,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.997,1975-09-01,true,771098089.016,17:6:26.51 +-1868529103,27466604483774576,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.997,1983-04-03,true,452315738.666,3:21:7.38 +-829199705,64522862414254848,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.997,1983-06-20,false,114531377.223,7:47:28.22 +2046070502,9447653914996880,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.997,1981-01-02,false,848651184.188,13:33:5.20 +758867655,13683721279217870,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.997,1999-09-06,true,680589840.487,11:3:8.37 +-13394861,39879136122714328,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.997,1988-08-14,false,10062867.9797,16:50:5.37 +-737653201,39402245679506808,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.997,1956-03-19,true,3167197.53622,20:36:47.22 +-895615750,56006477409208056,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.998,1979-03-25,false,46706664.8501,3:18:13.48 +-852484489,55614135545629688,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.998,1989-06-25,false,738644618.428,7:52:47.9 +-2136436261,41624180697034392,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.998,1987-10-26,false,668374547.933,4:40:5.4 +1233479051,65210470691763344,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.998,1969-05-13,false,873380107.506,15:29:35.42 +1453991458,69073192167983568,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.998,1974-06-04,true,166280778.87,13:13:42.32 +411652422,53786257514500024,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.998,1978-01-01,false,1085990513.33,6:19:58.14 +-1025401595,896185906053754,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.999,1989-03-08,false,336933888.025,2:4:8.10 +369566355,17854086589857034,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.999,1965-03-13,true,568438118.92,2:39:20.12 +565690199,4607249932187454,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.999,1951-09-17,false,1227337034.87,15:15:2.45 +-1423279291,90381370857561488,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.999,1958-03-12,false,168972205.412,8:52:42.51 +-917252922,65816922939930176,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:23.999,1977-05-21,true,683407988.829,14:44:49.28 +456640532,49486654181104312,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:23.999,1963-03-22,false,188618028.439,20:46:58.53 +-1301248302,15801952734838958,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:23.999,1982-08-26,false,138829459.808,3:3:39.44 +1424620717,67734156019061952,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:23.999,1972-11-09,false,1358900304.41,4:48:5.9 +1693875345,2474031298961049,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.000,1982-03-15,true,778644663.333,15:32:32.15 +-730366108,12029318452272394,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.000,1974-02-20,true,1431586893.19,15:15:37.32 +-1975677860,81035879497713552,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.000,1972-11-03,true,66860949.9587,8:26:17.14 +-215882382,81817042487562528,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.000,1992-10-10,false,1222660984.46,6:42:8.43 +-1521081786,91437427450763568,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.000,2005-05-19,true,993433540.629,20:34:12.56 +803076097,76045940770649216,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.000,2001-04-21,false,205826499.227,19:52:3.15 +-563733398,41494102372559024,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.000,1980-01-18,true,1365570760.36,9:54:32.50 +1780418385,22264725036778640,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.001,1987-03-23,true,556312493.375,14:27:16.38 +1762759787,11600999737942190,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.001,1979-01-16,true,614952780.05,3:31:40.3 +-1427474766,40490161195132304,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.001,1950-09-25,true,1152843428.62,19:41:14.19 +-1398845953,51058321514405040,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.001,2013-01-08,true,633498919.964,11:56:6.53 +-620476707,19002699198549832,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.001,1978-09-21,true,270649056.324,7:1:35.2 +-673150533,47900367505992328,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.001,1994-07-02,false,825568010.144,10:36:58.52 +1730020780,2087862079772948,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.001,2008-11-04,true,798924420.466,15:1:35.19 +1248498598,24234008472543672,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.002,2012-07-13,true,513891312.644,20:39:51.41 +-1962958670,57566471617029120,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.002,2013-05-24,true,834967291.295,13:26:37.18 +1160676922,36409862570793840,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.002,1990-11-14,false,1347410680.49,17:21:13.8 +2116544729,17766989547422822,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.002,1986-07-05,false,62756761.5866,14:8:46.29 +1018542952,89664703768800416,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.002,1991-08-16,false,433932059.04,16:18:36.37 +1383982149,41681722232276216,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.002,2007-09-17,true,897121167.255,21:37:22.3 +621128033,71732049932303136,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.002,2006-11-24,false,1120041196.05,16:51:4.17 +956804689,40142449659152008,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.003,1978-01-13,false,913668647.385,11:27:40.7 +588039965,26106238338536120,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.003,2003-09-26,true,197532094.948,21:23:15.1 +1592818341,36381400155200144,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.003,1966-08-20,true,112858974.656,11:3:30.11 +595930263,64070781206249440,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.003,1979-04-13,true,419993212.569,14:39:40.23 +1032464239,20842353487887688,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.003,1997-06-10,false,468385392.208,22:25:20.31 +202543242,43850698205925432,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.003,1975-02-18,true,109509857.235,12:47:56.15 +-679456853,79073935673987824,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.003,1968-01-01,true,1030409607.31,18:5:31.26 +-113875767,77586271022221296,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.003,1958-04-21,true,638020347.493,20:16:50.54 +-43427311,29741948184980328,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.004,1997-05-26,false,816441241.826,2:21:13.56 +131350435,91314347477908128,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.004,2011-06-09,false,1233339000.17,15:1:5.11 +-863652925,26625266619272860,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.004,1957-09-19,false,619302139.466,9:50:35.30 +-1559650347,11596256798118872,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.004,1983-05-25,true,831101244.865,6:24:52.1 +-509603745,33453809164186012,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.004,1975-05-23,false,416134029.382,4:29:52.49 +1331678583,8610600603296799,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.004,2011-05-10,true,826129376.665,5:6:54.39 +1458046046,44799643252100944,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.004,1995-08-27,true,347761715.224,19:33:31.39 +-1181021275,24029134200807180,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.005,2007-11-22,true,7667843.68439,3:44:53.37 +1807200459,85389038742285120,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.005,1997-10-18,true,1065921770.38,11:24:4.23 +-1438977776,34365702346939936,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.005,1952-11-09,true,385836501.862,3:42:52.43 +831023336,26749235805193728,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.005,1994-09-27,false,164899563.828,13:21:51.46 +-1524677671,80130559771794928,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.005,1972-04-01,false,433970233.937,19:34:7.56 +950818925,25101545630556316,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.005,1997-10-16,false,32941953.3912,16:25:31.36 +-1745742265,38237337767975064,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.005,1969-06-22,true,705716139.73,18:36:3.47 +1045142431,4133880413025976,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.006,1968-10-09,false,900798893.87,22:41:51.9 +132425115,40957288855593680,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.006,1990-08-05,false,1222503030.87,14:9:9.14 +1051977143,67305648112722784,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.006,1959-07-09,true,193104118.915,11:37:57.10 +1529875374,70512601634782096,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.006,1951-09-05,true,1070896364.0,7:30:7.3 +-521772289,7776523981917563,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.006,1968-05-20,true,648133969.495,11:19:14.26 +908270548,71736267877707048,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.006,1977-07-12,true,582423131.11,10:38:24.11 +465223423,13922516038342502,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.007,1977-03-09,false,146225393.184,16:27:45.56 +-435810650,29722233415091160,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.007,1987-10-20,false,1086262440.48,19:22:21.25 +-1606612090,50101654623438520,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.007,1960-08-08,true,556281598.561,1:7:2.12 +-1299840786,86171667159318864,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.007,1993-09-17,true,837334406.266,2:53:30.8 +-556618548,53722346155858392,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.007,2004-10-05,true,1194678681.01,6:22:27.25 +2008545360,86821150817607824,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.007,1967-01-11,true,542689425.777,2:20:52.7 +-1373685139,89785766475269856,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.007,1979-10-09,false,828861374.268,9:21:39.30 +-937588870,79208805834544944,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.008,1997-04-20,true,977419076.249,11:31:1.30 +14593830,14554688183660820,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.008,2006-06-19,true,2485623.31024,20:52:14.45 +-681153682,55875266466820408,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.008,1958-10-15,true,449786090.283,21:24:51.22 +8140790,22942751010059100,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.008,1983-10-13,false,999839650.802,1:49:49.31 +-1543440845,32126699490883452,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.008,1987-10-02,true,829334021.274,9:26:1.25 +-209383950,12753497757851802,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.008,1990-08-13,true,200500782.238,6:56:35.36 +375304150,18936610427767440,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.008,1950-11-04,true,400897266.066,15:55:10.53 +-423148427,44689389788980392,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.009,1994-10-05,true,42586339.3145,2:33:15.8 +673802569,86611592024336880,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.009,1967-06-04,false,576545137.909,20:51:2.51 +1470394395,39627443436865384,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.009,1967-11-16,true,171622405.023,16:52:40.37 +70194663,51036903279489648,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.009,1992-11-12,false,1287939535.22,1:27:2.11 +-1846887400,82118020323497840,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.009,1976-11-09,false,125006389.94,3:7:20.55 +228136832,34269012129455720,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.009,1960-09-18,true,314573255.631,1:22:6.56 +-463595703,19599196575041680,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.009,1999-05-20,true,32893027.0403,2:7:12.16 +-1464870676,33063747239631136,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.010,1956-09-10,true,1323435019.27,2:28:57.56 +542642106,35058099764076236,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.010,2011-06-07,true,932399299.618,12:29:41.50 +2020182609,89949659650409088,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.010,1990-08-03,false,1338256408.65,12:28:24.40 +-1244116719,37772172471310176,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.010,1983-03-17,false,614765024.57,5:54:39.41 +1974454050,32911746830911224,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.010,1966-10-04,true,57996895.4911,11:32:53.35 +-32823005,33267050905459168,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.010,2005-04-12,true,1194104392.31,18:3:17.7 +1508290208,32373560485182588,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.011,1990-01-26,true,272003946.583,3:6:14.15 +-33907751,49832113457411312,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.011,1966-03-27,true,528482716.488,1:43:16.10 +846720752,49337792731516536,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.011,1974-03-05,true,922585495.617,14:28:33.27 +-621234804,54999187618385032,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.011,2005-05-15,true,1022858852.7,2:9:34.56 +-348686348,710545583362232,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.011,1978-11-05,true,958685029.995,10:51:43.38 +-863085563,9010995284519106,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.011,2007-05-19,false,400103005.939,7:4:10.1 +-1215958609,70351398456952248,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.011,2000-07-23,false,155179040.146,5:58:39.50 +-2077704933,29891045486891868,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.011,1999-05-25,false,191090913.936,19:15:38.47 +-1632409933,78765121402286352,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.012,1970-11-20,true,781791121.967,19:29:20.17 +-1349981842,2723183376816148,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.012,1957-02-01,true,1018394163.81,11:5:1.29 +1698526607,47899743695178616,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.012,1961-07-02,true,1329284823.25,10:4:58.32 +849410973,20234235614398524,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.012,1989-11-18,true,174508163.577,14:13:29.8 +-392607113,77437307074810160,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.012,1991-07-07,false,138276244.388,7:49:23.53 +1666739740,8213834793568205,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.012,1993-11-11,true,837322326.418,6:11:21.27 +-1358722166,82037606256695488,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.012,1961-05-16,true,1432138704.22,13:37:29.40 +-1410112398,16818887781790076,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.013,1984-09-05,false,576037292.479,14:45:20.34 +130237223,60243934193105984,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.013,1980-04-24,true,7898501.10305,18:54:40.46 +1904424390,5791518926198006,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.013,2012-03-24,false,1165808052.01,21:38:37.17 +-296014828,30014976728568568,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.013,1981-02-19,true,806857992.489,20:6:38.46 +1142645433,69470929148147368,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.013,1977-02-27,true,224729703.696,15:43:22.1 +-1515702491,77236147717869168,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.013,1964-10-09,false,968747092.324,16:11:28.24 +-754590270,65848489028680256,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.013,2011-09-22,false,1419693174.66,7:58:46.1 +2046888752,47765200220806000,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.013,2006-09-19,false,1006227932.61,1:51:10.51 +-65476202,29725423836807088,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.014,1997-03-11,false,189593904.385,9:20:13.46 +1916007724,956123665451264,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.014,1965-03-07,false,1042877129.59,9:11:30.37 +1844853350,61299939145153384,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.014,1994-08-19,false,246930726.69,18:21:27.10 +1959950667,68856380917792504,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.014,2000-01-15,false,195533720.14,21:33:10.34 +621631411,22769851865584528,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.014,1987-03-17,true,1398085798.02,16:28:6.22 +-2047997640,67849645430526608,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.014,1974-05-22,true,828562641.528,1:57:50.29 +86800432,33591696002889116,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.014,2013-04-11,false,728844368.243,22:21:47.4 +-148855983,34532776618149172,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.015,1966-03-27,false,1171116850.1,8:50:31.42 +-139783125,54193738637381184,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.015,2007-08-01,false,1263743581.41,15:9:31.30 +2002420581,28887186783166024,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.015,2009-09-25,false,803431024.109,5:15:1.51 +1119996836,83008905542202816,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.015,2004-08-16,false,1120428633.31,4:46:33.39 +-1335969629,36056027846088112,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.015,1983-08-06,false,270676249.496,6:58:42.29 +2140314215,16412779744817542,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.015,1961-05-26,true,1022695419.88,6:18:51.48 +1737782221,78472311483799984,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.015,1965-10-18,false,867079664.065,20:13:48.7 +-2102736994,65071197714795672,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.016,1987-08-03,false,631378435.765,10:7:57.23 +-2136764722,46328409417876072,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.016,2010-05-20,true,1059689907.46,16:1:23.43 +1401257140,43069376501182160,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.016,2005-11-09,true,371395328.399,20:29:7.29 +-578793732,68967447917480304,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.016,1968-02-17,true,913009992.618,20:56:43.40 +1029384351,19831515175084360,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.016,1970-07-25,true,167183377.475,22:37:44.6 +-1044906041,82757082799124208,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.016,1993-04-06,false,120174362.578,7:6:52.33 +37392754,5260247630631608,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.017,1984-07-19,true,753837101.556,5:25:4.39 +1940181696,7325405136643830,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.017,1965-08-20,false,304679641.707,18:17:31.25 +814442822,31660255837889528,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.017,1985-02-06,false,743987604.046,13:8:10.5 +-1212101786,50417697102941720,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.017,1956-01-18,false,735429189.825,13:49:40.48 +1775736731,90721056613960816,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.018,1978-07-19,true,1431430175.58,12:37:50.48 +-675055546,5666462182100951,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.018,1983-02-24,true,1203342675.22,2:9:17.28 +-1842788325,90498674153741072,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.018,2004-02-17,false,1301362551.77,18:23:56.8 +-1558216176,56939532653733200,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.018,2010-06-27,true,1367141087.52,22:8:38.31 +1399125130,63345490743723816,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.018,1999-06-09,true,205293177.462,22:9:24.7 +-901309272,5868808289020764,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.018,1970-02-09,false,401742917.87,3:24:16.16 +-620982458,5745845772966646,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.018,1988-06-24,false,813036528.614,7:6:30.55 +-1752744994,78893079988298096,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.018,1987-07-01,false,1305112363.34,12:53:34.7 +1171343110,65215312938845320,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.019,1981-04-22,false,840299797.674,20:11:44.24 +1710541537,52231716982968328,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.019,1979-02-22,false,1356510800.89,17:9:37.39 +969638702,64909686802317776,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.019,1984-03-12,false,588988289.913,15:25:43.24 +-838220357,2152598259083755,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.019,1953-03-04,true,523523557.431,12:56:52.11 +1524806462,8346184459264041,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.019,1997-06-10,false,853837262.05,11:12:56.9 +1117958110,56025179671179192,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.019,1977-07-15,false,698415473.281,4:33:29.12 +1587958745,85621636437050288,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.019,1965-01-05,false,629424355.324,22:8:51.50 +1023400658,53422763447691888,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.019,1992-09-27,false,471631942.228,8:6:22.24 +1520216859,73260868388030416,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.019,2002-08-23,true,1303529190.0,1:33:33.20 +1190549708,69446134840150416,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.020,1956-06-02,false,114123290.656,2:27:27.14 +-543689513,46240315151863064,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.020,1995-09-16,false,452269056.474,10:48:30.33 +-438416198,9392989896056258,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.020,1951-07-10,false,818026978.811,6:12:35.32 +-11210532,10907900653725594,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.020,1980-09-11,false,107702936.839,7:1:1.7 +-770694957,63412708856428216,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.020,1988-01-15,true,1205122503.38,15:38:29.24 +1644129594,1690842748210340,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.020,1951-11-18,true,850797284.73,8:50:23.6 +-1209957713,14440865089923708,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.020,1981-02-12,false,683371257.009,10:3:1.6 +1633083628,86287477411792848,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.020,1996-05-22,true,875087990.554,5:39:44.55 +418379438,1203107043274915,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.021,1996-09-19,false,980317396.277,16:7:56.53 +1402676755,22410352594998620,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.021,2004-04-07,false,275078536.812,19:13:15.46 +-1252958270,81716710424699120,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.021,1987-06-18,true,688700130.891,17:20:56.3 +-582357198,37867604006914144,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.021,2005-03-05,true,15286461.7914,9:25:15.43 +-787714716,48665679345519176,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.021,2011-01-27,false,1066167035.67,3:32:48.36 +-83660010,24669271684531632,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.021,2003-11-13,false,993335276.168,5:46:12.32 +-588244473,53277287953000824,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.021,1983-01-10,true,655725735.497,19:17:28.58 +-1903595338,75139696656438800,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.021,1980-03-04,true,993989853.834,19:36:18.48 +-1684909095,92099514341934640,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.022,1965-02-08,true,802901667.846,3:21:33.58 +-1366933108,74705790372849152,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.022,1987-05-06,true,84859118.2265,9:40:5.27 +374302015,61973103874716344,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.022,1993-03-06,false,1261587725.25,21:8:10.12 +-1482763481,72217281517740672,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.022,1984-11-03,false,1021230412.57,11:9:17.8 +66327359,78884687509167856,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.022,1996-07-23,false,717929810.506,17:36:6.19 +-337563237,29343759543696948,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.022,1975-11-06,false,658357786.452,5:50:13.14 +-1277610482,44930107971840032,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.022,1999-11-08,true,939807958.903,18:2:58.17 +226369282,35461318346620344,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.023,1981-08-15,true,87940535.1419,9:48:4.57 +1134961877,50807348795738224,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.023,1984-07-05,true,1341314747.3,17:9:49.11 +1694521948,40592869648366728,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.023,1966-09-19,false,1226053966.89,19:54:5.17 +-1698735356,56949069932994672,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.023,1986-11-19,true,284408574.64,16:43:26.33 +311012415,59739993697786240,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.023,1999-07-20,true,762951292.169,21:22:25.8 +-1926386996,88554377328320448,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.023,1977-07-11,true,1381367075.48,15:11:42.23 +-101238165,46227997410573536,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.023,1979-06-25,false,91380392.8184,19:10:29.54 +-1963814819,12378668526046270,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.023,2009-04-26,true,984916241.514,2:48:41.14 +165506956,88803171756097552,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.024,1993-08-25,true,265940451.804,6:6:17.44 +1064088091,90852697230698960,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.024,1963-09-26,false,323501975.036,8:51:51.46 +241038239,27258951884648612,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.024,1976-05-21,true,432424112.918,8:24:24.21 +-1050432660,56727841383996864,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.024,1962-04-24,false,1036050831.71,11:42:51.14 +766485607,42435589994043024,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.024,1984-04-17,true,825033082.741,13:29:52.46 +-1929815014,91660182661272784,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.024,1978-07-25,false,1326378049.88,1:51:46.39 +-786675585,15643426638792418,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.024,1987-06-09,true,323766652.245,9:36:10.52 +378648830,7218707204512533,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.025,2007-09-11,true,1277298017.82,22:23:21.7 +592101622,6636234394841907,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.025,1994-09-25,true,1276458823.05,9:6:26.3 +-395113129,35630923145400320,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.025,1998-10-12,true,1095949058.71,22:22:56.29 +1764442271,15472520367937782,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.025,1978-01-17,false,502912425.197,10:19:17.8 +-79323090,19511365927177032,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.025,1988-07-18,false,939302030.833,8:17:22.39 +-628135190,32326315432848016,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.025,1996-08-12,true,697701939.136,5:58:34.49 +468788635,7521491720186225,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.025,1992-09-24,true,459608347.885,21:19:24.15 +1211052216,78595842523688752,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.026,1998-11-27,false,282316706.308,9:22:28.45 +-1446574118,33291856924405916,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.026,1952-04-20,true,1089722700.98,18:1:40.23 +824059295,47421623681524064,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.026,2009-06-07,true,1195543828.7,14:13:27.34 +-381096203,90710095754912976,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.026,2013-02-23,false,383914624.995,8:28:51.56 +-230199451,41311120837300320,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.026,2001-04-16,true,1097931655.85,13:27:40.25 +-2003494331,73070553447405680,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.026,1966-01-26,true,147163703.304,21:20:55.33 +-935614411,89359667865836624,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.026,1989-01-09,false,545475294.682,3:50:22.48 +375748547,23171838291385212,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.026,1989-06-15,false,1244707953.71,22:10:30.49 +182190957,48453690958280472,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.027,1966-07-01,true,52126046.1242,14:38:15.7 +1397821520,72449098654643440,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.027,2006-03-17,false,78737853.5211,4:33:54.43 +1783328459,29222348708457696,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.027,1953-06-14,false,222786000.001,5:20:52.30 +-283214684,20154121557259316,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.027,1957-09-09,true,290954231.133,15:2:40.48 +-555732031,27199237808305500,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.027,1967-05-17,true,255906174.507,14:54:23.29 +-1849005725,67879380284932720,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.027,1969-01-20,true,867747431.384,16:47:49.47 +-1697644805,59324345864629584,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.028,2013-09-23,false,515494843.607,21:20:36.35 +-672276442,40928627347316760,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.028,1972-10-24,true,735312639.047,14:42:11.18 +-1556010320,71854139126529792,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.028,1959-11-24,true,43060802.8162,3:34:27.38 +1596683200,6777512929883751,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.028,1965-05-27,true,646765839.469,15:44:17.54 +334722945,32593727749025064,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.028,1993-07-12,false,576810975.097,12:14:25.56 +-1368677536,57470579331672512,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.028,1985-10-09,false,435803954.219,5:30:46.29 +-463362014,5986574352116828,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.028,1985-05-26,false,856051721.38,11:13:40.49 +-1303978392,30915921437482220,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.029,1996-07-06,true,1126164013.26,4:29:36.49 +1296635255,73481615451115872,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.029,1972-08-06,false,422414012.617,10:26:17.27 +1150522071,38552416192688896,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.029,2000-07-17,true,160636751.786,3:23:50.16 +-1344770453,88246413643236032,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.029,1965-11-07,true,324323355.058,5:14:31.13 +151590553,41379993615054960,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.029,1969-02-15,true,1108392276.43,13:35:20.31 +-591691809,50310341486000624,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.029,1973-05-12,true,134707966.789,8:44:21.29 +40636937,33759190278374288,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.029,2012-03-13,false,1252524106.99,13:57:22.47 +1396339602,89614830521683536,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.029,2003-01-04,true,30240431.3716,14:47:9.1 +-1762292089,35458875634832036,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.030,1984-05-05,false,1022020502.74,1:41:33.18 +1280349215,76734848943499824,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.030,2011-10-23,true,1353671176.59,21:52:42.52 +1610641097,7930729060267131,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.030,1974-06-17,false,609213220.194,8:13:29.13 +69855597,46169613159485560,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.030,1976-03-09,true,1123598337.22,2:44:38.16 +1982276100,49501697628171616,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.030,2006-02-18,false,668582513.61,2:50:8.22 +-739786433,66824720222834280,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.030,1966-08-25,false,1103643790.65,12:5:4.4 +-85033586,88566570267427520,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.030,1967-07-11,false,1168284817.65,18:36:14.46 +-931808213,58646142053681288,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.031,1981-10-20,true,721721124.058,1:24:20.47 +-157379490,39764193055924856,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.031,1986-11-12,true,720516965.544,1:14:49.39 +-1071263203,29179994317338492,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.031,2000-07-15,true,151693862.705,22:25:21.38 +498689216,9811872040770160,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.031,2009-03-09,false,751347891.019,11:30:30.7 +-220538935,38145534953188600,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.031,1964-08-07,true,363144389.984,12:3:45.18 +-1926430123,2285021083335045,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.031,2008-07-24,true,267093479.441,10:37:48.55 +2035862602,47296702137742448,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.031,1954-07-27,false,714462029.706,17:33:15.41 +-100344872,39265829800118792,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.032,2002-10-15,false,865608115.921,15:2:7.16 +-1417005921,17642624558039778,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.032,1970-04-08,true,621506251.705,13:19:40.11 +177204672,50858102486923640,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.032,1980-04-26,true,647369924.813,6:15:2.36 +-2006500155,67974548828854112,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.032,1992-11-15,true,21544164.9728,21:9:20.15 +611632837,86791884844141504,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.032,1970-07-10,true,561292773.272,19:17:13.39 +188792892,31780016432408556,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.032,1981-08-26,true,759273191.203,7:30:44.55 +-1835614736,71379450748355592,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.032,2000-08-04,true,273349277.189,1:35:54.30 +-443254158,12655542647353068,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.033,1983-06-01,true,668062272.972,12:4:43.48 +1501147660,14395935276429568,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.033,2006-10-06,true,139638527.748,12:37:9.53 +-1353397223,3666565516665078,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.033,1954-04-24,true,638069183.28,13:14:53.39 +-918345954,89932001542524976,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.033,1965-01-21,false,414082496.476,11:7:37.19 +-1232529663,20694847996669624,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.033,1999-09-16,false,494449979.003,16:52:25.58 +1638821374,30356172622321448,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.033,1995-10-26,true,306028355.601,16:30:57.26 +-2071394165,61944790002910688,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.033,2011-03-25,true,1107575136.53,4:12:49.44 +966886267,71970951364139976,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.034,1988-08-03,true,485480359.166,12:18:19.4 +1407261877,59235813820577824,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.034,1975-02-27,true,1095279459.57,22:22:50.47 +2143073422,64623215090842544,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.034,1962-07-24,true,418740243.526,20:19:31.41 +-264104675,47425689326334432,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.034,1973-03-23,false,476905077.081,14:58:57.43 +-1555741115,39998962139573792,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.034,1972-03-20,true,1003271951.5,20:43:39.8 +-1100023093,46987021212377536,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.034,1973-06-09,true,812227208.19,2:34:31.49 +-17452787,28189337986508188,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.034,1997-04-17,false,812472645.023,20:18:16.47 +1173637765,4760280997402450,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.035,1962-04-13,false,1031611886.13,2:38:52.25 +-264295553,49660972645574680,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.035,1954-07-13,false,552017433.133,19:25:11.43 +1512913122,47783749356587896,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.035,1983-11-23,true,1290726876.38,6:44:6.52 +185913633,13528226133369610,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.035,1950-05-20,false,1388747832.7,7:8:9.5 +-1082058158,16372974592629340,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.035,2011-06-06,true,154114195.366,6:50:27.53 +-1915094585,55654734164185296,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.035,1984-08-02,false,20882754.9036,16:11:25.21 +387210466,3780707213076296,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.035,1960-05-24,true,781095117.038,4:55:58.18 +-301275485,88050523377131056,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.036,1963-04-27,true,1050675468.64,9:15:13.15 +344507913,8825935703298192,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.036,1976-02-26,false,1213608399.62,16:2:12.21 +1888139725,54126606886674840,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.036,1953-11-19,false,1229099106.68,6:21:14.25 +-1881323872,72505971107523088,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.036,2004-09-16,true,958790646.138,20:2:57.37 +-1335978805,4802810992230451,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.036,2008-03-20,true,1426338899.13,19:32:53.8 +122713751,17846166713251790,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.036,1963-01-19,false,1191401690.77,17:12:51.24 +-202541022,50563550456914752,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.036,1951-11-15,false,471970105.442,21:49:45.15 +-674109980,56872089353876864,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.036,1966-06-22,false,232638234.851,14:40:11.41 +-1153296877,27767046715965420,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.037,1958-04-21,false,573143575.899,15:10:48.51 +-1115432133,38620016385658592,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.037,1966-06-24,false,272316967.242,13:17:33.46 +-1142054,44245350491733184,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.037,1986-03-01,true,992772258.276,17:50:29.40 +2089052043,64186713384088248,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.037,2002-02-11,false,990653273.289,15:36:19.53 +658870386,19899298630515540,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.037,2011-10-24,true,46311007.5099,15:12:58.51 +-1163164889,62713850997898760,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.037,1972-08-11,true,1050347692.81,18:16:45.2 +1395776977,61653669747693736,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.038,1967-02-19,false,289309893.386,5:10:9.34 +817208982,35521154107986340,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.038,1999-07-12,false,613927754.227,13:22:27.57 +-571697254,87619346229044848,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.038,2012-01-15,true,20401890.0491,10:8:34.29 +-633256426,49540221115515840,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.038,1993-09-14,false,93610681.7928,9:18:8.33 +1636273461,35587083679392256,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.038,1985-07-04,false,639936972.563,16:48:31.20 +1994096473,21481813730173500,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.038,1972-02-16,true,1046530171.3,16:9:54.27 +-546158689,50113171772845304,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.038,2002-08-12,true,1376491145.12,2:40:38.13 +607397632,21883404252191304,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.038,1969-09-14,false,339638355.682,15:48:11.9 +654670513,5038094984131062,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.039,1981-05-10,true,107853690.578,17:44:34.46 +-1986268135,66124143757933376,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.039,2006-10-22,false,299453780.833,7:46:46.46 +1382684098,60889764001850880,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.039,2001-11-23,true,602259728.785,19:5:26.37 +435702044,5643058957260268,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.039,1978-07-15,true,120494840.834,6:51:17.20 +-1250279872,83129220627281088,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.039,1974-05-27,false,707286971.325,21:43:55.14 +1051421271,60082969911235216,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.039,1988-05-26,false,1298443523.26,5:52:1.56 +1418127508,3362354597274163,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.039,1973-05-22,false,1056881519.99,5:16:5.32 +-435678503,79847938104389280,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.040,1968-02-04,true,710956507.031,11:3:44.19 +256035479,43520461539913728,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.040,2008-09-24,true,95989603.7212,13:52:42.55 +834056572,52002184096118856,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.040,1959-05-02,true,618877510.998,20:19:38.20 +1316337482,10098218846342810,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.040,1965-10-03,true,717915.501561,1:32:42.23 +1338278915,63094509355804040,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.040,1983-10-02,false,166629463.645,9:17:20.28 +1218207658,84004333506968480,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.040,2003-03-06,false,353620932.205,14:15:17.32 +869307964,26848820596302920,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.040,2010-08-25,true,228026552.136,1:18:19.57 +-1149937286,56228355620682192,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.041,2010-06-06,false,375625365.613,3:31:3.34 +-1538963312,14428096212508918,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.041,1962-07-20,false,389519959.626,19:13:49.50 +426313526,15382884548194160,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.041,1982-05-05,false,1402405317.18,5:26:12.52 +1463011762,86881347004766288,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.041,1963-10-25,true,1164016325.41,20:13:24.22 +1626825412,86015298507545824,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.041,2006-03-25,false,1197991404.16,8:14:30.36 +138006174,7499768906577183,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.041,1951-10-09,false,82996804.5238,14:26:29.54 +4003980,40543784780597432,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.041,1985-11-06,false,655999883.112,12:42:10.34 +92708713,78570012601047744,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.041,1952-02-19,false,1087985284.05,6:44:3.19 +-137429120,36937666913953776,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.042,1952-04-23,true,381387765.222,13:21:52.18 +1034089503,26614555821399532,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.042,1978-04-21,true,1310351289.93,13:24:14.26 +642376076,47796243589417984,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.042,1957-05-14,false,584465415.715,4:27:44.40 +-2056621506,32169434461652492,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.042,1982-06-01,false,297834204.815,9:46:32.21 +-497419623,71933475145773920,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.042,1996-09-25,true,705810987.239,16:5:4.50 +-2040004306,40806028338851432,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.042,1995-03-26,true,1035600616.12,10:25:7.45 +1472551832,79153432152881024,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.042,2004-06-13,false,11745931.1098,3:57:51.2 +-1034212387,12770041015407810,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.043,1952-04-21,false,1365064513.7,5:45:30.46 +603653832,52947541633906432,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.043,1999-05-09,false,44495815.4441,5:33:4.54 +-2111841639,42711361972026040,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.043,1988-07-13,true,739627442.874,3:41:52.32 +683311323,91525175231061152,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.043,1954-09-05,true,622828127.083,3:1:36.12 +420957084,25848928543094508,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.043,1961-07-22,false,673862399.828,15:51:22.58 +-159929654,37964736457829568,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.043,1962-10-22,true,1204407825.68,22:9:33.50 +41259944,59151696194314248,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.043,2000-09-15,false,545302797.661,11:55:45.50 +1302362684,7055460510280039,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.044,1980-11-11,false,993068901.135,1:12:57.25 +-1425146259,40274763196676536,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.044,1953-10-14,true,1292699581.16,2:32:38.30 +-1246499145,17161532354038292,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.044,1998-06-11,true,4147719.70745,16:57:54.37 +-1256623057,35073698619346240,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.044,1996-01-08,true,1336811865.26,15:36:41.36 +-325087172,30294605327337768,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.044,1993-03-14,false,748059235.707,7:13:46.37 +453570374,31178855652865876,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.044,1968-07-18,true,719218322.103,22:58:19.47 +1897670909,10300389769588532,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.044,2004-02-09,false,976675934.676,22:51:24.25 +-1862901505,16834874482148988,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.045,1961-03-10,false,439203907.426,17:46:12.2 +-776127803,65723717056841904,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.045,2000-08-27,false,95457265.2918,2:56:35.1 +-505174187,63189467462938688,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.045,1978-03-09,false,1148435308.99,20:30:39.38 +-450469338,61610157436999296,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.045,1976-03-09,true,1187887511.02,14:1:28.53 +1396455036,57980297490738608,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.045,2011-10-04,false,944316212.041,7:51:58.2 +952099847,2972064582641889,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.045,1985-02-12,false,1137073452.4,18:17:14.18 +-1941448296,76066330178890400,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.045,1967-09-17,false,937280849.051,17:34:51.6 +243246827,30388196312093708,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.046,2004-01-18,false,1216922701.45,12:46:53.8 +-1506251265,77004358951427872,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.046,1977-03-10,false,1381211483.96,7:4:27.45 +-1333777083,70116864919132568,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.046,2004-04-03,true,242982363.842,12:8:39.48 +1007695359,57532281798059512,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.046,1960-01-27,false,454524832.322,10:42:54.3 +-1671666419,4952335154996808,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.046,1959-10-07,true,851065955.483,1:38:32.13 +-37056543,2833333456797706,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.046,1954-08-02,false,710663254.375,19:16:24.19 +973874753,71245061902412504,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.046,2002-11-10,false,516810178.738,18:51:43.13 +711278757,17820198686103726,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.047,1971-02-07,false,827273085.655,15:41:5.15 +315959704,73555023232588176,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.047,1982-10-16,true,1408848154.59,9:8:22.12 +809515693,63041179604636848,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.047,1991-10-25,true,760367119.177,7:28:26.27 +-886046529,50478345160072152,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.047,1992-08-10,false,1345690167.29,21:32:44.48 +-255287462,48015267695206144,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.047,1955-10-03,true,766475929.632,18:44:33.53 +-1725718884,1037563261188300,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.047,1956-02-03,false,1234026791.89,4:26:36.5 +-952025070,6476524289681449,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.048,1982-01-22,true,733331035.739,8:52:58.26 +-505787870,32147907896467148,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.048,1978-02-25,false,928093729.037,18:17:40.38 +1791518065,7302151110018734,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.048,2006-08-14,true,979970234.044,2:31:3.16 +1744198718,71536238290858112,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.048,1960-11-19,false,1237199211.54,21:8:52.31 +-1270657503,25674162688284232,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.048,2012-09-22,true,133303066.191,22:4:48.5 +-612023748,47390842967911336,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.048,1999-09-11,false,1109641609.75,5:25:39.6 +80496354,29695672802505164,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.049,1955-02-04,false,514106223.299,20:55:24.1 +-238089474,12746397685269340,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.049,1969-10-21,false,1112006064.8,22:17:41.52 +-1651135491,54695441426124416,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.049,1986-07-24,false,1420377071.97,14:1:16.25 +685682801,37232576543555152,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.049,1972-08-25,false,1168282860.98,5:16:4.5 +1328820634,61397347681426464,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.049,2000-04-01,false,847822740.432,22:24:10.2 +-2112477881,57882244851556816,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.049,1980-01-03,false,617140988.206,6:39:31.44 +2049801162,42016997527061352,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.049,1951-02-09,false,807734742.219,2:55:54.24 +927306427,50631758249564216,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.049,2012-01-15,false,1216937930.05,20:21:43.5 +582029501,53009141431837648,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.050,1998-10-16,true,951809084.74,19:8:3.3 +-1264722422,24712987656298652,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.050,2000-03-11,true,1239204828.85,16:58:27.57 +-830959419,7906388973786348,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.050,1966-08-24,true,984509123.888,2:55:5.41 +188211218,50236134303595872,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.050,2009-10-23,true,70550492.9528,20:41:47.56 +-1210756697,304463985569679,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.050,2003-06-01,true,57491672.1952,6:52:6.12 +-1975986175,31765252415412152,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.050,1994-02-24,false,409146851.432,10:49:20.40 +-826241235,30772705292675092,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.050,2005-05-08,true,692895684.276,2:19:14.2 +1274078607,6084661906639821,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.050,1951-02-04,true,1120635384.94,2:40:39.36 +-57560572,19906503714424064,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.051,2003-07-01,false,882491025.831,20:50:21.5 +1290437392,30173087299467592,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.051,1993-11-21,false,310378249.717,2:4:31.30 +-1478464881,91198829504223728,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.051,1981-11-07,true,1346868331.44,22:58:19.33 +1431449126,73181930655774848,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.051,1992-02-10,false,91619913.8078,16:31:19.26 +328965593,24848697863890780,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.051,2012-07-20,false,289213963.724,18:56:33.17 +632014503,82709508425857104,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.051,1973-06-20,false,249831291.158,3:20:56.18 +1607044692,4731074216358380,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.051,1982-02-09,false,988613793.811,2:41:20.7 +1964811887,73818026999686928,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.051,1969-10-09,true,424352016.858,12:40:33.19 +-254076736,18433323011018824,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.051,1977-01-19,false,912846145.088,18:11:36.51 +-2124911890,14335745559320566,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.052,1978-11-02,true,1071335242.49,19:31:34.47 +1059045457,86419229054597744,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.052,2005-03-15,true,1394301342.4,13:15:18.47 +1785865326,68888088867083856,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.052,1997-09-27,true,94971826.4859,7:3:26.54 +-1386904923,63847230698999880,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.052,1951-06-03,true,703271061.531,12:22:2.34 +81973422,17640525057372948,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.052,1956-11-15,true,3066641.96799,21:38:8.12 +-2126808551,30877491712361884,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.052,1994-03-12,false,258048486.73,4:16:53.33 +-1386345118,57291233805923792,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.052,2012-06-07,true,898658863.022,22:16:22.47 +-1727937738,53446158697984912,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.053,2002-08-12,false,501699853.535,20:43:4.42 +-1473722713,63491643749371824,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.053,2008-08-18,false,863636644.979,12:1:24.38 +262353539,6858260498306284,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.053,2000-03-17,false,863220698.435,14:33:29.20 +-1709945094,19017642298951208,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.053,1963-10-25,true,864301863.145,6:25:1.53 +-1811609838,7122295478964920,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.053,1976-10-09,true,878284187.933,20:17:16.24 +-1159994473,17634312706782996,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.053,1994-03-26,true,261699019.118,2:23:16.20 +132176211,37119209880661272,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.053,1977-06-24,true,727266972.056,18:35:18.11 +-1603743359,65406077368378768,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.053,1963-10-16,true,187035292.7,20:48:47.46 +-1746648477,9129498746768456,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.054,2012-11-03,false,548371234.074,10:24:14.40 +1776519499,89423058284606224,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.054,1968-04-05,false,1288541692.19,18:56:27.48 +-2026535837,16219134176882074,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.054,2010-05-22,false,790111812.669,21:23:31.39 +-1644322229,49552110250331608,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.054,1981-04-16,false,1050044250.19,1:4:1.39 +-1613166490,58774945636564064,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.054,1982-09-06,false,590329704.778,2:26:51.44 +996456594,10034684720078530,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.054,1958-11-05,false,1242003799.37,8:34:22.25 +-1640342447,91743602499193504,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.054,1979-03-20,true,836132827.35,20:15:17.15 +1092266192,6521843596514581,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.054,1969-06-14,true,459288481.729,7:29:54.48 +-1241020455,5725225686209761,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.054,1985-11-19,true,1199945347.98,18:30:21.6 +-555857541,78113464541370752,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.055,2007-03-09,true,873208449.02,14:16:23.2 +1623685533,62732431372501160,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.055,2007-09-21,false,933740404.328,8:23:40.8 +-1972599872,38642080055650888,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.055,2005-03-05,true,102585698.464,7:53:2.32 +-690349322,66173966188979992,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.055,1960-02-05,true,905729386.361,22:25:45.2 +-2014977774,78307160073278752,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.055,1984-08-13,true,351462321.672,21:58:34.6 +-1030108397,24653598082368596,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.055,1989-07-08,true,1048861302.43,1:33:6.55 +1054617289,59824806751612160,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.055,1967-06-16,true,707406886.595,21:29:29.16 +-41660459,24363736008054672,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.055,2010-08-02,false,1243362848.55,22:31:53.12 +363197255,6117182887987774,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.056,2001-02-24,false,198348545.946,13:16:46.37 +27103640,47736673368510856,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.056,2004-10-19,false,1368546062.9,2:15:12.48 +-1281075088,38464042196409840,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.056,1971-11-03,true,302773049.179,1:58:49.22 +1543055757,56435663407073272,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.056,1955-07-15,true,1117540598.1,3:1:40.32 +1350239836,11459891156783892,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.056,1974-03-13,true,1348782761.74,17:2:10.8 +770117518,24807799605232488,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.056,1964-09-23,true,1169621636.35,7:50:14.45 +1517037503,76456375079778272,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.056,1971-04-07,true,567420934.39,2:23:5.42 +-541559197,15849599527025890,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.056,2011-09-01,false,111399621.423,17:41:34.44 +-1818649329,14668423774980322,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.057,1983-11-09,true,109746245.858,22:6:53.34 +1933618208,30710406655822960,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.057,1979-02-04,true,339955066.025,10:16:44.1 +195450663,7595204468252375,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.057,1964-11-02,true,1142897033.89,18:25:55.18 +-1710195298,29359581366773800,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.057,1958-09-03,true,549075693.003,13:17:44.10 +-1798086662,86778678369961152,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.057,1991-11-02,false,465956100.622,16:24:42.49 +417120567,623018070786570,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.057,2001-11-26,true,182741736.86,21:19:55.2 +-125537194,91824125583106240,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.057,1996-11-13,true,1306876183.4,13:20:30.49 +-1587477696,44842360180426600,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.058,2006-04-26,true,416275499.559,12:23:26.13 +-43715525,25972047001473968,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.058,1982-06-19,true,934532883.899,6:43:14.56 +-1361346189,45344195439199608,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.058,1988-07-24,false,714850150.692,9:22:34.22 +-930322006,40840775949006776,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.058,1994-06-05,false,428607632.3,6:4:24.50 +-856101207,66998530934277952,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.058,1998-11-17,false,190685976.313,11:21:36.6 +1328830087,33264870656594472,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.058,1964-06-22,false,1341797750.55,9:25:18.42 +-1292171111,45726089813233048,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.058,1972-10-21,true,930031753.409,16:38:26.33 +-1458479423,74232975336900576,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.058,2006-10-21,false,1427821861.61,12:52:20.8 +-141078156,58077205098408128,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.059,1970-02-08,false,212494018.239,10:52:49.17 +1792682748,90768043296728784,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.059,1961-10-01,false,300193738.622,3:57:33.45 +-1776634674,71558647011620056,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.059,2008-11-23,false,83407488.8962,6:10:6.19 +1334509322,39329090708806912,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.059,1960-11-25,true,353165437.405,20:8:35.58 +-329274732,88753717251853536,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.059,1996-10-27,true,777732442.265,5:43:44.31 +1360307389,29168446411336776,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.059,1962-10-06,false,1009208775.62,19:5:43.34 +1241721316,69463884393748192,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.059,1978-01-04,false,344002494.154,13:41:6.18 +-729288561,82509861942430560,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.060,2010-09-24,false,807007212.243,5:49:13.5 +563690240,37073041171572664,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.060,1972-02-26,false,533852493.029,17:33:49.44 +1693856589,77655977784493024,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.060,1983-02-18,false,722243369.004,16:51:58.58 +1693313508,31593140604187648,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.060,2006-09-04,false,922255187.315,15:1:40.56 +247322223,7820501850856028,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.060,1984-03-25,true,829997389.276,10:29:35.38 +-1182361591,11424762493578190,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.060,1962-10-21,true,518848991.945,9:57:48.38 +-1412599961,53382754625769104,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.060,1951-04-11,false,50509636.3634,15:19:15.24 +-2070519102,31440116374427156,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.061,1968-03-07,false,1416717243.74,21:30:51.1 +-1481405500,43749243972455192,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.061,1970-10-03,true,691374138.665,5:58:42.52 +-115829332,32414646009400660,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.061,1967-01-21,false,610323843.388,6:39:7.1 +-319452581,69230358074293288,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.061,1958-07-09,false,842884856.94,13:1:7.7 +-1172658150,44225853428324936,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.061,1982-06-21,true,399137252.436,17:30:23.35 +347279760,71131958050362552,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.061,1962-11-13,true,621398694.803,3:4:25.19 +-628309597,41191186849891544,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.061,1975-05-27,false,219816668.562,1:53:57.58 +1227237864,60263527907007984,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.061,1962-08-23,false,281764261.511,1:8:49.16 +-896812748,12107830140800030,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.062,2008-06-01,false,624217289.888,17:43:1.57 +438976210,19368200142723164,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.062,1997-08-24,false,440711238.612,9:6:3.24 +-97749557,34467769893443236,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.062,1990-03-02,true,379362791.677,19:46:35.57 +-834961794,46072052125954496,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.062,2013-06-12,false,985687760.407,16:8:41.49 +1649676956,8482647546905426,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.062,1954-01-13,true,899152689.435,2:55:32.27 +380348825,71215539236178240,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.062,1982-10-02,true,1012638053.37,17:54:44.37 +634205821,20016132528348928,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.062,1955-08-07,true,68941163.5897,3:37:45.32 +1088397578,16245275429912280,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.063,1952-02-14,false,1351382058.95,13:6:1.9 +1649115205,76289024552728800,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.063,2010-02-05,false,978049996.089,22:34:50.3 +-51106836,27938321126314580,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.063,1967-02-01,false,472693717.161,5:39:40.22 +-1764533036,58305301085922816,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.063,2010-10-18,false,782209074.291,14:53:35.46 +1426274877,50969057179672984,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.063,1985-09-01,true,1398498639.0,15:26:22.19 +998643572,84915910740311280,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.063,1981-04-14,true,366709290.649,7:30:20.43 +-695121142,18622513195756564,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.063,1974-01-05,false,1360478320.26,6:43:51.47 +1028718797,70632363585956224,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.064,1962-03-09,true,587197378.297,5:1:45.14 +981331303,57956446893442368,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.064,1970-03-22,true,501807225.987,1:10:50.36 +-1302134608,83614842846363712,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.064,1994-08-23,false,201452828.048,6:51:45.15 +-1635237832,65542915230349552,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.064,2012-07-14,true,271914564.631,19:53:21.40 +-2064073460,75725257519890160,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.064,2005-11-18,true,93391490.731,10:30:26.4 +-741092453,84131715759590736,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.064,1975-04-24,true,682126094.857,9:32:4.46 +-1966993579,36481008127988160,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.064,1990-01-05,true,306275777.813,16:21:26.10 +225900169,72391097446535120,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.065,1988-08-06,true,527766585.774,16:7:40.32 +-462842376,39839489004692000,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.065,2005-02-14,true,928345782.325,13:18:40.3 +2116013109,4076555753699512,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.065,1955-11-22,true,108112019.191,20:23:47.6 +-856360981,87788916380219712,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.065,1950-04-23,false,1259611281.77,1:5:55.15 +1510942125,73883830771330144,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.065,1961-08-15,false,100707618.211,10:42:18.37 +-1761968051,25489328324621272,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.065,1987-10-15,true,1131437973.4,20:21:40.51 +-1368016761,75450555832007456,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.065,1972-10-10,true,61611576.6373,14:25:34.45 +338849046,18751856885894136,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.066,1958-05-03,true,1028979325.22,11:43:5.22 +1107013567,67900307264518112,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.066,1980-10-10,false,60233180.5592,5:38:42.52 +-1740980128,43735915250087360,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.066,1980-08-17,false,1236073198.21,11:38:54.3 +-1090150438,20952134802548008,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.066,2012-07-22,true,1005724220.77,9:45:37.33 +-68219624,63878260200665840,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.066,1978-10-26,true,509099133.093,14:54:7.8 +-1266115064,70107445338149632,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.066,1952-03-08,false,1155240410.88,7:4:54.22 +268548368,43049314017728568,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.066,2004-11-23,true,312401484.012,19:26:33.36 +-1447460847,62330543837756624,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.067,2013-08-01,true,969155322.658,10:48:27.51 +-1736735179,16403802831022880,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.067,1977-07-04,false,578962494.099,5:45:49.14 +1693044734,92046410275760560,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.067,1950-05-14,false,551387379.142,13:31:12.14 +-1406091852,17065356731945554,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.067,1983-06-05,false,701641895.63,15:50:31.9 +-2131784327,83814355826799120,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.067,2000-04-13,false,461522662.505,18:16:27.47 +-957253241,81465346052777456,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.067,1955-08-23,true,937500754.058,9:57:2.47 +-257049560,38173801733844480,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.067,1964-05-09,true,1267754922.12,16:23:27.52 +60060456,66314968336979632,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.068,1960-09-17,true,959921683.006,8:4:11.49 +-173207291,24435867347008780,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.068,2003-06-25,false,1269638169.4,22:26:21.33 +-1364252081,30023521204136132,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.068,1950-09-05,true,984099971.288,19:58:37.46 +1769768985,39642132748440320,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.068,1951-09-12,false,558907351.165,4:48:4.34 +1721015051,79024379553967920,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.068,2010-10-17,false,557574509.699,4:50:52.32 +-39935962,75062120294941200,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.068,1976-06-14,false,473672462.664,9:53:36.54 +-1800518804,82781472085961456,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.068,1973-06-16,true,159801391.249,10:36:18.33 +154965753,12285346678066556,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.069,1987-09-13,false,406412511.689,19:52:9.38 +546159735,82629494804631888,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.069,1963-03-12,true,327494586.331,17:40:44.4 +-1055610271,79670805032919712,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.069,1959-10-01,true,229015067.433,15:17:9.28 +204982515,71745457242166640,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.069,1987-07-20,false,1006747498.3,19:25:40.22 +-1678657875,29897315931439484,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.069,2011-11-13,false,563165039.214,11:7:8.23 +187287537,8933279633170053,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.069,1972-02-06,false,542051112.736,7:28:1.17 +-318361784,41213750783851088,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.069,1967-11-08,true,1380709912.83,2:53:44.15 +803641486,78412926959026000,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.070,2000-08-08,false,277186235.476,20:19:22.25 +-1014785689,76819680177780384,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.070,2005-06-08,true,1275891865.6,21:9:36.17 +1206500777,35844898289507092,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.070,2011-09-05,false,1099232474.8,6:8:21.20 +1075078472,31060319232611648,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.070,2006-05-06,false,257743933.167,16:16:3.58 +-534283109,1265441837539901,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.070,2013-05-06,true,427862760.135,12:40:33.24 +1118060429,2819441589161318,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.070,1988-03-04,false,1215863905.2,11:34:10.18 +-1551145446,33320076147245496,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.070,2009-05-23,false,1422791157.82,18:25:39.36 +312581144,79962289697943328,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.071,2011-07-26,true,1415605132.24,18:12:12.16 +-2084471393,83957578997526368,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.071,1991-05-10,false,1402074078.14,14:43:1.12 +-1814371732,46501895738649368,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.071,1958-08-05,false,583311492.697,18:15:36.37 +-802349128,30046351183139984,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.071,1971-10-12,true,728528326.575,3:13:46.13 +153679268,23259473055759328,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.071,1975-01-07,true,407666176.327,11:11:4.25 +-1580584720,65781926091366760,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.071,1981-07-06,false,1391859737.63,13:20:3.9 +-2033574050,54741890727043176,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.071,1967-05-04,false,1050299641.91,18:30:7.3 +-1852243930,2753621849055938,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.071,1986-02-01,true,1341379662.4,7:53:6.41 +1045443585,18136701073148940,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.072,1970-06-09,true,628932610.566,7:18:24.53 +909803821,64101516714400536,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.072,1990-04-05,false,39976431.4412,18:52:20.41 +1652079127,402600997936844,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.072,1997-03-14,false,559254391.768,15:19:34.51 +988966424,42254140135943920,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.072,1995-04-26,false,958551599.861,6:46:58.27 +-768262191,51822691507548056,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.072,1973-09-03,false,104070300.919,9:16:58.40 +1487757501,87265572562482160,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.072,1960-02-23,true,91831212.1542,20:24:52.3 +-345058158,26311004096893084,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.072,1955-02-01,false,974985688.521,7:27:58.45 +425035417,28301776571977420,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.073,1968-09-07,false,91747528.5298,3:36:48.28 +673369290,36357303782235272,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.073,1958-05-22,false,1427990615.83,9:46:1.20 +191606089,73503862478177296,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.073,1968-05-16,true,220047140.656,4:10:43.48 +-949973965,70006460463348152,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.073,1961-07-13,false,807792057.708,4:27:28.25 +-1959881648,72966000929256672,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.073,1966-03-24,true,718160082.332,21:21:29.29 +1143444744,37001670996542824,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.073,1959-10-23,true,55101548.5078,17:17:27.12 +1177120860,19926659484855820,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.073,1950-04-16,false,313735697.88,5:45:55.51 +-284194735,30552164301214208,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.074,1965-05-16,true,942266283.294,14:30:4.45 +-984524672,75880128471004720,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.074,1981-03-13,false,136632935.95,19:6:23.52 +-1791235284,39420485137383248,AZ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.074,1986-06-13,false,315784499.092,22:35:25.28 +899533748,38678675658108008,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.074,2007-04-26,true,536296292.124,16:22:40.12 +-1887777207,5540583427737856,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.074,1956-05-17,true,1135540731.92,20:17:7.52 +1796324234,9798792848011132,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.074,1986-04-06,true,265143329.406,19:31:57.5 +622121952,5342737681638584,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.074,1950-03-18,false,395588258.884,13:29:24.25 +-1753568479,81859110405415760,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.075,1980-08-20,false,242001267.316,18:34:11.2 +1944501385,76494655827539920,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.075,1997-03-06,true,1030287615.58,7:32:31.29 +1734887828,52561725678158672,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.075,1981-02-06,true,1167905633.15,22:7:9.1 +11074347,83330088465320944,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.075,1992-01-20,true,235996655.835,1:49:36.40 +-183692289,16981601605652900,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.075,2010-11-05,true,952233805.094,22:42:8.26 +1107443919,7400379359817953,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.075,1991-08-19,false,481096119.97,7:31:45.31 +-1860768139,82486868165731696,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.075,2004-07-27,true,151262290.788,9:8:12.47 +755461358,52021912012371696,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.076,2004-06-17,false,208071578.945,21:24:49.20 +1071564028,58800623525150504,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.076,1969-01-12,false,249232112.484,2:17:57.51 +878219634,2583216079278756,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.076,1981-10-25,true,285472639.57,18:4:47.49 +640953329,20072696362324612,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.076,1999-08-25,true,595115038.391,1:46:34.57 +1514706863,731287440542873,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.076,1975-10-05,false,296952025.68,15:48:22.18 +-259207287,24165827772895212,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.076,1970-10-08,false,1196430018.26,12:38:20.58 +1629632256,45503026622650736,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.076,2002-08-05,false,189631287.875,11:48:27.48 +1622991610,84686321860760624,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.077,1977-04-22,true,918443740.161,1:37:18.36 +994408084,23608690110378476,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.077,2011-10-02,true,819097819.463,2:49:56.23 +629501828,22927146238193080,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.077,1976-02-14,false,172318878.231,7:30:39.50 +1725793483,34066655922143528,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.077,1969-09-08,false,282844046.029,22:40:12.41 +472536318,30498740280871672,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.077,1991-11-05,false,435552413.236,19:20:48.21 +-381829970,60084353447556528,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.077,1973-04-15,true,547357092.223,15:27:18.29 +-997059677,4623402660263967,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.077,2004-11-26,true,221885725.008,18:51:7.54 +-628862004,56655944980986808,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.078,1987-02-22,true,1056467359.69,12:49:40.38 +-515759347,16582212741835060,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.078,1989-01-22,true,456312069.655,14:29:49.30 +-1627319960,73136973637004192,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.078,2001-06-07,true,670693640.889,22:30:32.6 +1774136060,26992115175039712,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.078,1991-10-04,false,1050448764.24,12:47:7.46 +1808884011,75267984580321856,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.078,1963-03-06,true,411812569.661,18:24:25.15 +-2125877143,17669177828389714,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.078,1983-09-14,false,703577609.745,21:45:6.39 +-109103210,50377252728685240,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.078,1989-04-02,false,585231224.962,9:56:46.31 +689266549,89567581366077408,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.079,2011-04-20,false,309108907.165,1:20:3.38 +1760928615,21427380092016444,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.079,1956-05-11,false,505752520.368,1:43:43.13 +1360289792,39756749827470200,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.079,1978-05-19,false,330556229.092,5:16:17.43 +1065754390,70045635681318688,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.079,1989-10-13,false,156082963.654,13:41:56.57 +2103182399,78325386154144880,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.079,2001-04-19,true,1220383783.44,6:56:47.27 +-1560914237,16129147071201240,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.079,1988-06-16,false,219268587.618,7:21:13.32 +-141774829,28809586209071392,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.079,1978-01-03,false,267271513.327,15:34:13.31 +-2085866754,27198371243548560,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.080,2001-02-12,true,304167519.31,13:44:36.7 +244753185,5348199887847076,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.080,2004-08-17,false,1224131869.75,2:15:47.30 +550518030,21902497838235268,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.080,1992-03-06,false,1355144462.15,8:30:2.13 +-986104570,61469132509874176,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.080,2011-11-04,false,224509934.064,9:1:34.27 +23054159,49005107148702424,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.080,1962-05-12,true,1011652923.15,8:13:24.31 +-1176032310,8058205605849201,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.080,1951-09-24,true,79468357.9538,22:25:27.24 +-1476088492,1648457673573929,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.080,2009-04-17,false,1105467852.08,20:29:56.48 +-1858886949,48421990386838696,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.080,2010-09-01,true,277715751.047,21:36:58.24 +-1208146945,91533340714167536,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.081,1963-11-20,false,260168816.661,8:16:23.30 +-810281518,9787547295863010,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.081,2003-11-10,true,8816055.39378,22:33:28.32 +307093466,46449171832803728,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.081,1967-03-03,false,495113212.021,3:15:17.9 +1395297176,75739394048349264,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.081,2007-02-21,true,640701093.421,12:7:29.49 +-1384373255,71898513767824424,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.081,2005-08-27,true,649677732.846,1:42:15.46 +-237045510,83227244249071872,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.081,1965-01-07,true,214926654.812,5:41:51.24 +-1500366044,50295856769363392,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.081,1994-06-22,false,1231322.45987,9:54:23.36 +1815829876,37207365985254568,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.082,1999-10-17,false,846450890.474,15:21:34.22 +1497740524,12233102922178550,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.082,1993-04-23,true,145832741.087,15:26:47.55 +925284784,53401782477336928,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.082,1952-08-18,true,87756498.7548,6:49:15.6 +1311226969,1399534705663580,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.082,2011-08-08,true,393381426.479,10:7:7.27 +1613202826,62400271326655552,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.082,1966-03-27,false,1097934632.31,4:32:52.13 +-443827881,86521552603832688,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.082,1960-05-06,false,1286617494.68,15:43:31.10 +1238514113,86375057083927536,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.082,1956-08-26,true,298573153.176,13:22:43.36 +-822073772,88554376179365760,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.083,2011-11-06,false,229923132.947,18:46:39.32 +-1126518447,27435151237713224,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.083,1970-05-15,true,922548242.417,16:15:26.9 +1151771342,90746175870890368,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.083,1988-05-25,false,96671394.2577,8:4:42.1 +627103786,55666928735524640,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.083,1998-05-21,true,672974748.835,1:24:15.27 +-1853579073,17732385140186482,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.083,1997-07-20,true,1097380123.51,3:50:23.9 +1579480596,72529414380405776,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.083,2011-06-05,false,691304495.552,14:47:4.11 +-2007505610,68267989304094200,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.083,1996-08-08,true,468841410.398,20:39:30.3 +-459087229,80704643957488000,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.084,1986-07-13,true,724803252.291,9:33:35.22 +-1939887440,4032351302020383,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.084,2007-01-06,false,1393014982.6,12:51:18.29 +74720686,11454318011329700,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.084,1981-07-01,false,466115230.159,16:3:41.10 +1423528686,19048380384417372,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.084,2005-02-01,false,653474498.146,19:19:55.48 +453947621,79882902319643456,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.084,2009-09-13,true,1183474282.52,3:3:55.42 +-282235039,88182434894171472,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.084,1982-04-12,true,729509159.686,19:17:48.38 +-1921670023,74027293226346448,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.084,1955-02-14,false,274317221.276,10:31:1.1 +-30378533,38126349158708440,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.085,1985-03-20,false,706484343.249,4:33:15.53 +1427898305,50151600627643432,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.085,1958-08-25,true,849343641.798,18:22:40.42 +-2074623414,13619931995627356,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.085,1981-06-11,true,886016040.988,20:7:33.26 +149660755,15478486062032098,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.085,1988-09-07,false,1282496662.79,15:50:9.53 +-1505639321,79039273792197200,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.085,1976-07-25,false,1253831289.16,5:27:6.27 +1394156623,5583845036531200,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.085,1961-07-10,false,1420903049.52,15:48:39.34 +560335185,48389991873792952,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.085,1957-03-01,false,921965822.621,4:4:4.5 +388901067,66346562434096056,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.085,2001-05-08,true,35699932.0064,17:7:13.38 +-1101005547,80049177364231312,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.086,2003-11-23,false,1207731995.78,17:37:25.10 +1770098430,6789034539020626,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.086,1971-09-09,true,1268349017.48,8:44:46.34 +1750590088,76771739515336608,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.086,1974-10-07,false,29267915.0405,18:40:8.5 +1016140874,77510282540523936,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.086,1950-11-07,false,67091234.5627,2:7:51.44 +1036868624,57330512881669896,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.086,2010-01-18,true,236793422.588,3:32:45.18 +431020148,44837832076732584,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.086,1972-01-16,true,335660822.865,20:47:43.22 +-376364575,19313844257214608,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.086,2007-02-21,false,72981701.2904,20:43:15.39 +1866489294,4909040097087294,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.087,1964-08-02,false,361887177.481,2:45:9.57 +-1344276393,88433337847055296,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.087,1973-09-18,true,1088468321.25,8:58:45.41 +-1735035600,1708703822040033,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.087,1993-10-06,false,38014337.4716,16:1:48.54 +-1510161656,22822000467180380,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.087,1960-05-12,false,85965652.067,22:16:39.20 +-826066002,9984879006604668,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.087,1961-01-23,false,562750169.401,7:2:40.27 +1364432436,10161144426254080,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.087,1992-05-03,false,150225748.249,16:3:53.43 +1015750717,23205843009102768,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.087,1981-10-21,false,449705795.412,21:1:33.24 +-342167482,30111842588386480,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.088,2013-01-11,true,125614109.199,16:24:45.43 +-1034776464,25660664785192960,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.088,1992-06-26,false,1411166161.65,15:21:53.13 +-1130317924,58060028766219912,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.088,1983-10-27,false,108643600.308,22:51:38.25 +2037023791,15645161156528394,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.088,1992-06-21,false,73728512.9216,5:49:16.22 +-1254139603,40073242288297904,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.088,2012-02-24,true,1019561062.81,4:13:14.3 +945966655,81910441059172672,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.088,1962-07-05,true,258748893.174,4:16:9.45 +614583022,47487014659766784,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.088,1962-02-02,true,1425358607.79,3:12:15.31 +1125649859,12728660590701854,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.089,1994-01-26,false,1270114373.32,20:9:33.48 +-1929435892,38957474471603624,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.089,1997-07-13,true,381915685.975,22:36:43.14 +-1232093031,72217014824405888,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.089,1964-07-18,true,96987984.8912,16:49:35.37 +122297785,6624596275719189,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.089,1977-08-05,true,723148241.534,6:16:35.44 +-160736277,3641766412515113,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.089,1979-03-04,true,304500466.21,4:41:49.11 +-629970611,20730777428445104,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.089,1991-02-20,true,502836989.885,5:16:6.26 +-914709368,7918292481378089,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.089,1990-07-23,true,609808802.22,19:23:9.37 +-444653993,24008139370013728,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.090,1975-04-25,true,1306824221.58,7:25:6.17 +-688742281,4346967759267051,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.090,2012-05-19,true,1129258743.91,15:44:15.58 +276913979,81108498258741216,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.090,2009-07-18,false,668191223.729,10:42:20.15 +2120257966,3151771592432824,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.090,1952-09-04,true,891152445.616,10:45:19.34 +-1972574208,80171335770456944,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.090,1982-01-10,true,27755779.1452,16:19:32.32 +-338371883,13358036384654746,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.090,1987-07-07,false,552747879.477,5:6:57.29 +42808186,62388912700167376,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.090,1972-06-08,true,173763329.53,14:48:13.23 +1748999099,7160830177348711,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.091,2004-06-08,true,1132682794.29,6:49:57.35 +-1048526014,85646644940944096,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.091,1978-02-21,false,257511878.111,12:14:20.51 +-591215589,41938712454513904,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.091,1972-06-23,false,563112093.834,7:10:45.58 +1536148299,22551075489365648,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.091,2001-11-14,true,1058739905.12,16:11:22.57 +-336400986,31087881655798824,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.091,1977-03-07,false,1040229465.25,3:52:31.19 +1806360148,68660198991232368,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.091,1981-07-03,false,331465735.271,3:29:30.28 +-1304757331,86224848508126576,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.091,1955-01-10,false,871810255.997,14:42:58.41 +457875601,58979895309593144,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.092,1959-09-22,false,1305900859.33,4:9:10.55 +1047141887,16502534916962080,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.092,1984-07-10,true,952829655.893,20:26:1.49 +-166784949,58645338660750632,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.092,1969-09-16,true,231778901.189,9:43:5.39 +-565992040,24065287636136060,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.092,2007-11-05,true,452260461.306,5:30:44.27 +1774027129,8124632141178983,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.092,1966-07-04,false,1072399583.71,3:9:13.16 +-566560874,8894621420779612,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.092,1965-04-04,true,315393677.068,12:46:23.29 +1568614455,90914880545227280,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.092,1970-03-08,true,1006875215.59,10:11:49.3 +1436765025,8737977953334231,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.093,1987-10-01,false,435868583.357,11:49:1.3 +62624290,84415928783080640,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.093,1990-02-05,false,1080782642.39,4:3:50.47 +-1612996106,67728314384019928,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.093,1966-04-02,false,310097227.977,5:41:1.28 +1923580398,56302482279415872,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.093,1976-08-23,true,602939356.955,12:51:16.25 +-823680983,27401737222783272,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.093,1969-09-20,true,454407185.946,18:50:8.19 +-2043091721,62632630704376344,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.093,2012-09-24,false,1205326335.04,9:41:19.36 +-1346511780,85077958665044416,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.093,1982-02-26,false,589364921.057,16:27:16.57 +190405232,63175293706447472,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.093,2008-06-25,true,438549752.833,16:26:44.1 +-1745971283,64456255954761168,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.094,1983-07-25,true,458887379.518,14:24:58.5 +853564779,23077643361209008,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.094,1980-01-22,true,1379484404.5,21:28:31.31 +-556301089,27562038693163428,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.094,1977-04-14,false,182444206.08,12:58:29.1 +773603795,10032735280257894,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.094,1970-08-08,true,113117111.752,22:53:34.24 +-1418306527,39336099020401088,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.094,2003-03-21,true,40216363.943,12:1:49.52 +1637626127,79948652460256016,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.094,2007-01-13,false,103390004.833,18:36:46.41 +2064222750,10939098968908308,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.094,1971-10-10,false,998708214.721,14:37:43.37 +2016886142,83923849169205184,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.095,1961-01-17,true,680877334.822,2:43:9.6 +282058970,89125360851722832,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.095,1976-05-21,true,994597489.601,6:36:55.8 +939581201,57873151897322432,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.095,1989-07-19,true,196428360.001,19:30:31.37 +-757104305,22074641889452932,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.095,1983-11-14,true,271643519.184,10:6:37.42 +-666484549,66917467300431640,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.095,1974-11-22,true,990235351.817,10:29:35.33 +1316225741,12881233965901332,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.095,1966-08-14,false,1216504286.45,16:39:19.13 +251617495,49544987389753584,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.095,2005-05-14,false,312067157.308,22:37:28.18 +-470065245,66486629903691904,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.096,2013-10-23,false,543993264.372,6:29:14.25 +-1369921886,29247421592329144,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.096,1979-11-21,false,609257456.255,17:27:6.7 +-1657645772,53612167134709904,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.096,1963-04-08,true,621446642.421,1:58:33.23 +383400379,73340895781828448,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.096,1989-05-03,false,947201499.099,10:46:48.47 +1986627063,49826142600926168,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.096,1954-11-15,false,200845489.726,14:51:4.47 +883472428,51305634030777480,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.096,1991-08-02,false,378241560.46,14:23:56.9 +-1728658135,50845197190240704,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.096,1995-07-15,false,1109961319.07,14:37:47.6 +-1215325093,58625954827125360,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.096,2008-01-14,false,614318437.271,21:11:33.35 +940763691,18184607962218016,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.097,2006-08-04,true,1012667229.17,9:15:51.4 +2050306612,71552413220167672,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.097,1991-09-18,true,921882518.171,4:48:49.31 +340384029,31079278732363532,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.097,1965-11-21,false,1116502263.43,19:16:44.13 +-2120628231,45228328173807552,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.097,1974-04-07,true,1428864091.41,21:36:38.39 +-748384141,58693316662340448,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.097,1995-04-23,true,702439004.391,14:7:54.11 +783019875,59197985222816264,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.097,1966-02-27,false,663825605.456,6:43:12.55 +1428126945,20216366414307000,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.097,1952-06-13,false,293551083.189,19:47:56.51 +834824718,18974776367307880,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.098,1973-03-25,true,331869488.175,20:56:41.42 +1072911723,79297320893266928,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.098,1988-11-21,false,190040264.864,21:38:2.41 +-1392823464,51871623435938272,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.098,1983-11-18,false,1269321333.9,9:34:33.57 +-1479055853,89139642454499840,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.098,2005-10-13,false,243620482.641,18:35:37.29 +988949966,28241284282366752,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.098,1980-06-18,false,1097830194.18,10:25:50.7 +519223925,3562602029068298,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.098,1970-08-03,true,1127923410.8,5:39:35.28 +1377429890,67442218957263288,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.098,1990-03-08,true,571891591.57,7:41:4.31 +791524369,64996221827334848,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.099,2012-05-17,false,1137585480.9,3:2:55.29 +646834270,19648268775728140,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.099,1955-07-06,false,565581511.163,16:52:5.19 +-1064403839,48744370844541848,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.099,1965-02-19,true,1354421799.71,21:32:52.26 +821584396,17560992398867324,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.099,1989-11-18,true,1235067747.3,14:36:38.15 +-1031878816,18165208333894996,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.099,2005-11-27,false,1368969208.46,6:32:54.35 +642833772,45869140027200680,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.099,1981-05-22,true,1072510701.64,1:22:13.25 +43194716,59277614987270512,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.099,1968-04-18,false,25972809.6807,19:12:28.10 +-1860175418,65838060464663912,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.100,2001-10-07,false,1307022725.5,1:4:8.53 +-1321190188,2044274362013010,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.100,1958-10-03,false,1206630412.44,1:20:13.37 +1612257924,41999014451359416,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.100,1973-07-09,true,212210274.799,13:50:51.29 +-999916565,72770352395293280,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.100,2009-10-16,true,1072695755.84,7:24:57.54 +-912612052,40324188869182976,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.100,1984-04-07,false,1224762321.31,9:5:29.39 +-1884859443,85443216088966672,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.100,2010-01-12,true,1383988191.44,9:57:32.10 +-2136133714,42305905529787832,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.100,1978-02-09,true,199952522.487,22:20:36.41 +496560308,66825069317761248,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.101,1983-04-02,true,283222849.949,19:18:24.38 +1377394705,8970117175943097,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.101,1960-05-10,true,340666552.614,5:29:7.31 +-928809171,28879109147346964,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.101,2009-07-15,true,8156208.54342,8:40:33.8 +1175390022,28335748080206428,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.101,1988-09-15,true,90897745.8784,6:32:47.44 +-1065987999,43390814291498296,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.101,1962-01-11,false,1056789245.99,17:31:2.16 +765164020,32280558358474272,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.101,1981-02-25,false,1053839203.84,15:1:9.27 +-1922531870,46830183801416048,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.101,1997-03-11,false,558871140.21,4:30:39.40 +-1721221584,55140173772121184,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.102,1977-09-01,false,21728091.6267,14:49:51.45 +829081703,84196294765978736,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.102,1968-07-08,false,927072093.888,8:48:30.46 +415924895,7508817344720589,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.102,1963-06-04,false,1294025921.38,1:47:29.44 +411947522,82210342984233232,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.102,2006-09-06,false,503920615.475,11:51:28.12 +-1015614736,36030526795498136,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.102,1978-07-19,false,1274639705.27,2:39:19.31 +-2047360712,40141041278563056,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.102,1978-01-15,true,626343168.812,15:51:44.33 +532231040,8950032461640868,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.102,1994-08-06,false,566413153.073,13:34:55.32 +-1738020081,90053921879392720,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2002-06-02 00:35:24.102,1960-02-20,false,1025271893.75,18:51:32.13 +-1716429912,45174726928125584,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.103,1979-08-23,false,1374794311.9,10:47:54.33 +1063188842,5913575449057393,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.103,2013-01-19,false,1084353899.27,5:25:53.56 +634831413,80471543923543536,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.103,2013-11-10,false,805387194.399,8:5:20.41 +785961658,65093010820872440,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.103,1987-11-09,true,514638854.632,6:45:26.56 +-1678186062,91814193401337984,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.103,2010-01-18,false,1101253585.1,11:46:11.2 +1713294443,52765038868601520,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2003-06-02 00:35:24.103,1979-11-05,false,1007546775.29,18:8:31.54 +-1580494585,43427745659400056,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXJ,2011-06-02 00:35:24.103,1963-11-25,true,403078460.838,17:50:12.15 +1456873819,50036103933776504,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:35:24.104,2008-01-15,true,1189647914.07,1:4:35.51 +302567114,40159891131406960,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.240,1960-08-25,false,1397060877.19,16:44:44.50 +493866955,23236996231610296,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.240,2009-09-17,false,882898530.625,14:3:48.3 +2106997910,36372608413463048,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.241,1983-10-18,true,368625020.924,7:3:38.29 +-1850242434,71758766206541880,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.241,1994-06-26,true,1063530510.98,19:24:34.47 +75893329,41176744192222240,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.242,1968-01-19,false,116958114.566,1:16:14.47 +-1883107423,50191136925185496,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.242,1983-06-11,false,806159854.907,7:53:11.57 +-376889118,62065701461420160,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.242,1958-11-26,true,1425615866.39,9:18:38.38 +-819204794,79313294351922928,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.242,1981-03-15,true,1190533104.29,1:49:55.54 +1397965220,21865766656367832,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.242,1964-01-15,true,1402526362.46,20:38:34.48 +1587450297,82812551569327280,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.243,1977-06-06,true,5620721.82795,3:18:42.37 +1606504273,40724047138521496,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.243,1993-02-23,true,285830702.985,16:45:47.11 +-1153446568,75124467955598896,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.243,1994-08-19,true,11947337.0339,3:30:6.3 +573606945,69653869174548984,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.243,1980-07-14,true,754003990.743,4:56:5.45 +-286339989,11582573376059064,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.243,1955-07-20,true,489862252.595,10:10:39.48 +-1499326585,35708755207986892,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.243,1957-05-22,true,238760992.057,6:52:40.1 +867891277,34840942132828068,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.244,1986-04-16,false,147950440.61,11:9:32.11 +1030322188,86329275008787504,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.244,1994-01-15,false,1405357976.67,15:57:17.15 +-770644021,6873169186819635,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.244,1981-07-07,true,858526191.366,16:13:46.29 +854736294,85085040851251216,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.244,2012-03-26,true,78721850.5257,6:25:34.53 +-855902742,77978941510425472,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.244,2011-10-26,true,1315959857.37,5:19:38.9 +-676405486,24574789481246148,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.244,1961-09-23,true,191096818.733,13:11:21.36 +-1908816558,43207211718750360,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.245,1969-06-11,false,567000099.027,18:36:6.55 +1234220541,15642334322854840,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.245,1970-07-06,true,904198977.491,15:8:21.17 +517480591,860773697581701,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.245,1968-07-25,false,668727269.068,22:21:47.8 +-1349702613,58815546792379416,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.245,2009-01-10,true,1407192888.63,17:23:21.25 +-752179912,26847457536864736,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.245,1963-03-19,false,164283462.995,9:16:35.22 +1041450030,29362357503946372,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.245,1959-07-08,true,411233549.622,21:36:39.37 +923816510,24228141636796232,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.246,1997-02-11,false,602388428.091,19:48:46.41 +-1309474812,81677115595913840,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.246,1984-08-10,true,307323393.52,3:3:53.31 +1825042955,23021938772538296,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.246,1995-08-02,true,683343414.22,17:13:5.12 +-108965556,77470433620465168,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.246,2006-11-22,false,166239540.37,8:13:40.31 +567923410,4804589723025971,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.246,1971-11-06,false,385403418.324,9:55:42.10 +-1652440712,62887236966920504,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.246,2008-10-12,false,378627519.564,11:53:11.37 +-1662310308,3312603320720947,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.247,1975-07-16,false,109434733.79,1:17:55.41 +-359638212,86475509108162784,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.247,1950-02-17,true,1178994071.1,9:49:58.1 +-1163600453,68418083232820816,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.247,1965-11-21,false,1124210762.43,3:52:21.21 +1858307970,62386190793140672,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.248,2008-01-03,true,453152580.906,13:23:27.28 +-137855311,10089454924400774,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.248,1984-09-12,true,823378995.872,17:12:14.42 +91249530,10971622854515528,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.249,2004-06-06,true,155016033.807,17:14:2.3 +92989856,56652268237418152,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.249,2012-11-19,true,945110586.435,16:29:45.7 +-1141173969,2512775584112578,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.249,1973-09-05,false,1115780994.29,3:24:23.54 +1672034882,65913479776757048,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.250,1962-03-11,true,1285139497.63,15:47:49.40 +1501047819,29391757568349696,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.250,1996-01-22,false,54779694.0689,1:13:5.52 +460456025,67046811281585760,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.250,1974-06-23,false,1357651015.18,18:28:39.21 +-2114719180,75751460508037904,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.251,1985-03-17,true,613816488.273,10:43:30.41 +1006604911,57662419288790128,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.251,2002-05-05,false,1200256239.78,9:23:9.5 +-1883108082,63405505313277072,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.251,1987-03-26,false,940121604.074,4:52:21.48 +1515625860,19023287625089832,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.251,1986-07-01,true,28586232.9797,18:31:14.45 +-1525939273,10030120989518438,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.251,1973-05-04,true,87706042.8798,3:17:38.51 +1337527241,2033151518533202,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.252,1975-07-21,true,43393220.7102,21:49:42.50 +1823914335,63043867092610048,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.252,1970-11-19,false,896600939.082,10:47:30.33 +39683245,46310997177592680,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.252,1953-05-19,false,815899686.776,1:6:45.37 +601527478,58076616441466000,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.252,2008-02-08,true,181441386.309,9:23:8.21 +-242208942,32773877278862900,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.252,1978-08-19,true,1101963029.95,20:35:7.11 +95608566,24358794166262212,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.252,2005-04-23,true,653849674.636,3:35:39.37 +2023745255,9030710024846244,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.252,1987-04-24,true,1272965305.03,19:27:14.30 +1717574048,78132554552179760,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.252,2013-07-13,true,941674659.31,18:17:9.16 +-619730879,17230132961052846,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.253,1989-11-24,false,922186767.901,2:42:17.18 +355416089,65752031350317336,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.253,2013-09-09,false,677316167.542,11:7:56.43 +2072967655,10484282324508580,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.253,2001-02-17,true,959674178.671,21:3:18.10 +-992350415,89204267782700848,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.253,1960-02-14,true,976983532.276,11:22:12.31 +981239113,63023438897583272,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.253,1959-05-12,true,1398762792.49,2:44:53.16 +-1879350083,90001989734127424,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.253,2004-10-06,true,1008084585.58,14:51:45.4 +43434891,12342169659747882,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.253,1997-01-26,true,1341530126.09,3:54:25.45 +856456881,79692664955500320,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.253,1957-08-16,true,303594889.643,18:14:33.34 +-335921284,18125272757544324,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.254,1957-02-03,false,219758075.075,9:50:31.9 +1819824998,6363614409257953,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.254,1954-08-03,false,144621797.646,18:50:31.26 +-1821267448,5082665021933701,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.254,2006-08-01,false,1216424197.92,21:54:13.3 +1846842237,65501644626143592,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.254,2006-03-03,true,277767469.547,11:39:37.20 +584046955,78547977938423680,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.254,1957-07-14,false,1409078413.25,20:15:39.35 +1197608127,42359673923090528,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.254,1975-01-25,false,716956094.075,7:22:22.42 +-977631210,65280156858859176,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.254,1973-03-04,false,1358658278.8,9:28:41.12 +-580212372,35398680109762836,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.254,1998-10-14,false,1193459235.86,16:10:56.42 +492675559,34708333256088548,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.255,1986-09-16,true,858091551.129,4:1:15.44 +923896660,7379169889858693,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.255,1990-05-11,true,510533385.338,1:11:25.32 +-1618660493,8561687317318482,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.255,1965-03-23,true,1186993584.48,9:53:39.1 +-1261199232,53874660592625584,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.255,2010-02-04,true,684061252.79,12:43:43.9 +-1919415405,20095692314530612,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.255,1959-02-21,true,649689270.277,14:2:36.37 +68015332,18088283981286768,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.255,1959-09-22,true,1255513817.88,20:34:40.9 +1940448280,81794334179599264,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.255,1974-08-22,true,994943982.376,4:7:42.55 +-293959785,46943748180712736,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.256,1979-04-27,false,459249513.408,16:39:36.36 +-1714234812,81141896131337344,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.256,1980-04-04,false,594117916.039,16:35:50.14 +872707006,83162224606683088,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.256,2003-01-01,false,1041907194.22,9:46:27.4 +1371221216,60950216667386816,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.256,1953-11-14,true,1382558037.19,7:23:1.14 +851360474,81718978134621152,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.256,1969-05-19,true,1285416570.39,1:50:44.33 +-1812534039,18453094362020752,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.256,2009-11-22,false,1395248367.58,16:5:32.37 +-1275350800,76694172640044624,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.256,1963-08-02,true,371084810.203,6:5:39.34 +-1462606849,67950256462310024,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.257,1983-05-18,true,220039979.774,15:40:32.55 +-924458942,20312524019591372,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.257,1959-05-01,false,622875474.76,19:34:43.29 +197258649,89382973756665984,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.257,1977-11-19,false,1033560809.36,19:47:37.15 +-319843834,45352799037821248,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.257,2007-09-26,true,1230972497.84,14:56:13.33 +-1675310764,7301428870189896,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.257,1994-09-01,false,594792681.272,9:23:38.55 +-145992731,71251472445144536,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.257,1997-10-27,true,995843458.271,21:9:58.33 +-16386377,10250905504104398,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.257,1969-04-13,false,578496242.411,1:4:32.55 +-802279298,58601646824321456,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.257,1953-08-21,false,932636351.523,5:41:10.35 +-45304974,8725752722536796,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.258,2007-06-16,false,258700770.936,5:21:13.56 +1856323610,14553679880372942,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.258,2003-07-01,true,552245670.853,7:58:5.11 +1579300857,66433309160993304,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.258,1990-10-22,false,814137249.324,6:54:46.53 +-624542822,48882781703874384,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.258,1967-08-09,true,904790939.677,14:8:36.15 +-159054771,30143643121193400,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.258,1973-03-09,true,1309680266.63,18:32:21.51 +677569707,16465029471649096,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.258,1974-10-16,true,611130397.373,20:47:40.16 +1632420328,7050335325318953,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.258,2006-01-15,false,1038970644.67,12:53:34.20 +-887911506,43957443338371504,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.259,2009-06-09,true,1192166812.36,22:55:58.9 +-1619360772,33828713455208716,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.259,1966-07-20,false,1384860521.53,9:36:1.58 +264686374,9339664805878968,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.259,1952-04-22,false,877858690.181,21:3:41.48 +-76688061,28426595097229272,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.259,1958-07-16,false,1158479613.25,5:22:50.10 +-2100339591,32191567414184100,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.259,1955-05-13,true,761417022.257,8:38:15.38 +-1749815388,45249718931467336,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.259,1954-08-04,false,100757967.837,1:13:5.32 +-1870187166,51414846789769464,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.260,1983-09-13,false,1380069840.23,5:51:18.55 +-355841652,39039853861317368,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.260,1994-04-24,false,1311442113.72,9:7:34.19 +1350987168,31189841562054616,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.260,2000-11-27,false,323041793.918,21:7:32.30 +-256304106,59667130157800448,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.260,1961-08-08,true,892807315.584,21:29:35.47 +-921699736,14645122879116350,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.260,1995-05-19,true,937057211.975,6:39:8.2 +-1997073803,13649832078242242,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.260,1962-02-17,false,1397523888.38,6:9:33.36 +1999539163,84692167643087968,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.261,1953-11-07,false,734130842.197,14:47:44.36 +-275384070,80935557068244896,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.261,1997-11-16,true,1176809533.77,3:10:48.10 +1732019083,29101848351817492,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.261,2010-06-05,false,922452437.762,17:3:11.19 +-43093181,76433324157233200,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.261,1965-10-23,false,38583411.2247,9:53:21.1 +-1413038625,70434771831573768,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.261,1992-08-16,false,369739469.726,11:3:43.30 +-471699836,34586340919634444,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.261,1999-05-26,true,1249014268.68,12:50:54.44 +1972772481,58542462765520264,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.261,1969-02-07,false,258310893.76,2:38:48.36 +-543012613,20862002168527544,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.262,2012-08-15,true,755603215.208,5:43:21.8 +1061233885,46569928287978344,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.262,2004-01-02,false,300676068.429,5:1:6.56 +-32057086,15351944069103708,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.262,1989-08-03,false,421700258.165,13:15:24.14 +-1456496009,63346644168218664,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.262,2008-11-23,true,1215393991.36,9:25:1.52 +-2022434613,59925228105356904,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.262,1984-07-16,true,1058994685.54,8:39:46.14 +-1413548484,39327789591746016,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.262,2001-07-07,false,1427742022.04,3:23:36.13 +-837658786,63022418362692392,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.262,2010-01-20,true,826305234.217,15:19:6.23 +1858638199,67847644891797064,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.263,2003-09-03,false,503097584.331,10:34:53.55 +1682843162,86456681928613024,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.263,1975-06-10,true,215055578.369,14:58:20.17 +2009691442,31773381398356060,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.263,1962-06-08,true,540266603.804,8:28:45.56 +683785531,32710986045796556,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.263,1987-06-24,true,1101178625.58,12:1:35.32 +-127928895,36344414457714392,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.263,1977-08-27,false,255110229.331,20:14:56.51 +2093952113,31506746810625300,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.263,1960-02-02,true,1115331676.1,13:28:21.53 +-1571418303,16395970799029730,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.263,2001-04-09,true,129943020.943,7:40:9.8 +1591649273,48327525800984584,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.264,1964-07-11,false,172526476.791,20:2:11.56 +188601711,50367998109754448,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.264,2010-07-08,true,1174541009.24,16:58:36.19 +605703942,32850787129010800,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.264,2011-04-26,false,830668858.902,13:47:34.42 +-511592381,49324974832144416,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.264,1978-08-06,false,1311136681.99,9:52:20.48 +-1197032184,63361976536153808,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.264,1963-04-27,false,298614448.92,8:32:8.44 +645166657,19971851467246716,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.264,1951-06-10,false,1088333795.4,14:8:16.54 +-97051454,87313296874274688,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.264,1981-06-21,true,1218955313.31,17:27:45.3 +-262110563,92131972330148176,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.264,1986-10-06,false,716203223.066,1:7:38.11 +-2108176924,50769881541469960,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.265,1989-06-18,false,120092475.487,21:1:26.41 +-1971846127,55156056265062544,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.265,1971-11-21,true,614542945.027,6:13:41.30 +1583532684,70059272201765368,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.265,1959-09-07,true,1323828635.11,2:15:12.27 +-649448930,15200548287602114,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.265,1977-11-11,true,1091168941.59,5:43:26.25 +800888786,54081711486607280,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.265,1990-02-04,false,1218264809.86,22:8:28.1 +1255334238,73433427353903248,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.265,1998-09-27,false,1139653396.76,3:47:22.38 +1376829618,56382720070683168,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.265,1979-03-10,true,744044777.399,14:4:21.46 +-879413293,53810802361283640,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.266,2002-11-05,true,880332072.384,20:42:15.11 +385816794,64427477113644480,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.266,2007-08-08,false,766392124.152,10:1:31.18 +103661924,53770311827588472,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.266,1992-02-11,true,49329145.3069,18:56:47.22 +-1887701212,72229734515401808,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.266,1971-03-07,true,762402508.645,16:55:45.39 +-1637104706,44974907702464800,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.266,1958-01-25,false,441327930.389,17:36:4.5 +745456302,20874726483866972,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.266,1966-08-11,false,378303278.36,16:58:18.32 +-1548714583,17371472122116558,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.266,1990-09-23,true,822889120.762,2:16:19.27 +-159267844,67158466262977064,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.267,1978-04-10,true,340414322.315,18:54:44.55 +451642424,69144272561080784,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.267,1970-05-26,false,537991497.702,11:46:15.10 +56825807,63093276973517280,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.279,2006-01-12,true,1382616908.5,2:28:49.36 +-1182183280,71653076998962440,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.280,1952-11-05,true,430138165.519,12:14:20.51 +-368489964,41493550047852088,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.280,1995-08-05,true,1126863758.12,10:22:32.10 +1810209091,63245755157137664,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.280,1962-09-23,false,1140227919.42,13:28:11.13 +745229272,66515502967120960,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.281,1988-10-09,true,103266805.601,22:22:35.11 +-1548497429,25161507699564400,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.281,1978-04-12,true,804653262.21,21:17:8.35 +1760466136,17206267236890614,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.281,2001-11-09,false,1090128882.56,4:45:20.53 +683068122,32334333734761484,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.282,1968-09-18,false,639039170.732,17:47:36.14 +1463426614,23488113156828456,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.282,2005-06-17,false,500108226.014,3:57:23.17 +306951340,85132445974432784,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.282,1994-05-01,true,549383551.4,4:44:2.23 +609467809,52994293566206304,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.283,2006-11-23,false,1622488.23943,16:34:3.25 +-1982420269,22451864210836448,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.283,1988-04-19,false,788184492.954,12:45:3.1 +-1430793604,55166469503949200,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.283,1975-09-06,true,406435080.847,9:50:22.17 +1847118642,22964786804302920,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.284,1953-07-25,false,227837473.758,12:52:18.15 +1441286537,15314078514991402,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.284,1960-04-18,true,179080788.065,2:2:19.21 +-1871850090,13832276095663852,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.285,1988-03-07,true,1218448906.64,13:28:10.52 +-1894235079,67163398480231464,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.285,2009-09-11,true,479363367.581,2:25:52.49 +1776847467,54792542166568656,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.285,1950-09-11,true,72759789.5649,3:54:10.27 +-1043230642,10846209766928210,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.286,2012-09-14,false,171813018.481,12:1:7.30 +1713949563,51399513068563936,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.286,1966-05-20,false,53804258.5371,18:39:21.13 +-634915884,68493484516188712,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.286,1970-05-07,true,1054350156.51,21:49:39.56 +587742440,80862225313930816,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.286,1992-07-22,false,275612235.178,6:3:36.29 +820711200,14316119130230928,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.286,1960-01-24,false,553760939.011,19:11:10.20 +-266681425,31706106785661224,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.287,1969-02-20,false,1134294099.28,5:32:33.55 +-985134547,60866558252202712,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.287,2000-01-07,false,372230284.611,9:49:30.5 +2092544615,66745078603897776,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.287,1954-08-05,false,112178539.578,3:50:33.41 +-996610364,63999460754848448,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.287,2006-05-12,false,1125442376.59,10:11:36.50 +-942777885,1206630674083123,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.287,2000-04-10,true,187067774.027,16:2:45.26 +-1166901380,36658290298164928,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.288,1991-04-19,true,437014354.288,11:7:15.27 +1339556594,81771689839168816,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.288,1967-08-26,true,628842474.039,18:24:41.5 +1563312752,39632896215139808,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.288,1961-04-02,true,1369160598.35,20:46:10.2 +-1044593870,69764229747578800,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.288,1995-05-23,true,235026670.368,4:3:54.17 +-1705131476,14667194236665856,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.288,1997-10-13,false,362753035.152,18:58:44.22 +-1946325431,8955040401329388,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.288,1967-01-03,false,585483829.583,10:17:55.10 +990224480,16125552309363692,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.288,1991-09-03,false,916971460.227,5:34:16.38 +-1476057677,72015301773872128,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.289,1966-11-21,true,1259499448.3,10:55:24.42 +622420424,15293037233675920,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.289,1960-05-13,true,742003007.547,2:1:57.12 +124964619,83345735396086576,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.289,2010-02-13,false,655130559.848,5:33:44.14 +-1642045158,91931003864300864,IL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.289,1966-11-19,false,560732286.391,20:5:45.37 +1485351806,27748274805900840,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.289,2010-11-26,false,517365424.391,18:11:1.34 +-1593542795,61330196771386464,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.289,1976-05-05,false,603075494.2,4:34:23.45 +-736299256,22529938365888708,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.290,1969-07-03,true,961325011.37,14:58:19.19 +-9325481,49455745316754328,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.290,1960-08-25,true,324820866.669,21:41:32.39 +-448451544,1192495321037987,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.290,1975-05-27,true,1169710793.61,16:22:46.7 +1923306490,18620643897975336,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.290,1964-03-07,false,33728071.1101,15:4:50.32 +-2004761227,54761895618614264,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.290,1996-08-06,false,575453421.034,12:11:35.25 +2123589878,4212974865240084,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.290,1973-01-13,true,486571279.289,17:43:34.35 +301257330,79244792805192288,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.290,1956-04-21,false,329594213.05,14:8:35.39 +-1912314915,39484171794994464,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.291,1954-08-18,false,661043462.209,12:18:36.20 +439711314,20900933398171956,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.291,1954-01-10,true,765697696.205,7:50:14.22 +-755771288,33393117711467376,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.291,1962-08-08,true,1275009226.01,10:56:21.18 +-522605106,7947592396761866,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.291,1998-10-10,false,1288196332.72,15:23:8.50 +-122588594,57212663487302056,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.291,1994-07-06,false,542990161.799,3:44:53.29 +633599635,76571367988346288,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.291,1962-06-07,false,254096901.725,10:7:54.48 +-1505192378,11624506663203790,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.291,2011-06-15,false,1025086317.63,9:37:50.7 +-1611056038,39395023164408648,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.292,1964-03-15,true,216582627.353,15:33:35.15 +-592864515,72888774720113792,WY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.292,1990-03-21,false,510382594.912,15:37:17.20 +-1672346012,36544251939636856,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.292,1981-10-24,true,578165280.273,16:3:26.11 +-713452593,60134200540584808,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.292,1990-11-23,true,152523823.624,16:42:56.6 +-1918492885,88415767305638160,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.292,1991-10-02,false,869913541.942,3:22:57.7 +-811589634,89036160579560448,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.292,1975-09-07,false,806877871.169,10:32:28.50 +-1391182,3318330733717156,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.292,1986-06-27,true,294409644.158,15:11:13.7 +-1101864075,58084325751761184,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.293,1974-08-02,true,841035780.608,21:37:23.17 +-1887926057,64917226609662104,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.293,1982-10-11,false,1201820711.66,21:14:37.33 +1294815561,76763755232541712,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.293,1973-05-07,false,444694893.173,19:12:6.33 +1349061378,41148019797124936,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.293,1993-01-02,true,98886243.8609,5:12:14.36 +1090350088,30068456827624828,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.293,1958-04-07,true,695536796.689,3:44:8.29 +-1586823584,39408557360380784,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.293,1993-06-03,true,154277512.463,18:58:37.22 +-1133515179,13220924467824896,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.293,1962-10-17,true,1154836012.4,17:43:41.24 +292361708,4572060130427136,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.294,1989-08-17,true,1260646136.67,9:4:50.18 +1681423867,5616957845883351,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.294,1957-02-04,true,1230218518.92,20:10:46.46 +1982367203,52497523380311584,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.294,2011-11-04,false,964131860.429,6:24:32.40 +1615130814,37036145443830224,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.294,1977-01-06,true,389294102.85,13:9:53.40 +2093266971,81676079702295568,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.294,1999-04-25,false,713544330.494,20:49:27.5 +-810378848,49898771650757368,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.294,1982-08-12,false,223119589.841,3:41:20.46 +675386418,56562536927083344,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.294,2013-09-15,false,928831652.423,5:42:50.11 +-1702953481,16961230624941998,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.294,1953-02-10,true,1194485612.28,22:10:47.18 +-997555521,63838042264746504,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.295,1978-04-18,true,967598524.818,16:5:56.35 +1163269665,80517369619329296,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.295,1995-11-06,false,898687089.602,20:24:29.44 +734854453,34337635179429592,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.295,1967-06-03,false,1012449297.29,11:4:46.21 +-596535639,65097074898254816,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.295,1961-02-08,false,1350285191.07,3:3:24.17 +1896449676,23363931217712764,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.295,1975-10-04,true,1134028725.03,6:21:5.20 +1457455742,68930228381424440,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.295,1952-05-18,false,13396700.82,21:33:19.58 +1466476198,17880656526005842,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.295,1972-01-27,true,883275856.408,1:7:52.9 +1507556952,5490185041510881,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.296,1956-11-20,true,1064466120.6,9:47:57.16 +-1994664773,13751513033346878,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.296,1978-10-03,false,604529859.066,12:26:56.54 +-607014885,18440438535115604,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.296,2000-05-25,true,1331201203.74,5:15:52.23 +525558154,51756189669633672,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.296,1978-05-04,false,1311368090.32,10:18:4.37 +-756939652,77982279228141344,OR,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.296,1981-04-01,true,175458558.268,4:2:57.36 +-134298608,86952875497126048,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.296,1989-09-21,false,89982223.7954,13:42:15.41 +-1258243228,27121172169744028,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.296,1975-02-02,false,662222964.35,8:39:19.32 +1105048039,27580647943474472,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.296,1999-10-18,false,751416352.523,12:58:37.5 +-872151538,41768766690765968,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.297,1963-10-12,true,1076128153.91,11:31:2.7 +7395771,40392383480273248,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.297,2007-06-02,false,604159522.91,16:39:36.52 +-439675492,7939951143357614,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.297,1995-06-09,true,146686159.471,15:25:12.10 +-1838721029,4847062307241370,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.297,1997-03-09,true,253414554.847,22:53:19.50 +338178744,14206131290515814,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.297,1980-07-20,false,65523773.5262,22:43:14.4 +645883087,10009188605889240,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.297,1990-06-26,false,1114049129.71,18:7:43.58 +575522243,50434999512710496,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.297,1960-03-05,true,809702927.633,15:42:15.40 +650061900,90208314458471696,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.298,1959-09-15,true,1041414254.69,11:45:53.6 +-471288923,49063567178265160,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.298,1986-05-16,false,725094614.96,2:18:45.7 +875242956,34976075764246008,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.298,1959-07-10,true,564882588.072,20:30:29.7 +589900402,87119636560511952,WA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.298,1997-05-11,false,1115056360.66,20:8:44.39 +399573375,4392188962389739,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.298,1992-06-15,true,442368971.188,13:25:20.27 +2034293399,79693098205696160,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.298,1971-11-19,false,1004844377.28,17:8:21.37 +1125784231,53884947673078928,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.298,1989-06-25,false,1277055767.03,5:23:49.21 +-983659534,45383977137382512,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.299,1950-04-05,true,518684785.172,16:3:1.35 +-352727989,8611690534544036,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.299,2007-04-09,true,211336430.265,20:34:25.41 +2080841960,5193119408976773,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.299,1985-01-07,false,1080058201.59,4:1:25.1 +2104691408,49185544307830024,IA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.299,1968-06-22,false,1105224714.16,20:39:31.54 +-287120157,28007194165759008,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.299,2004-04-25,true,682420369.457,22:24:53.6 +-1064477858,8682491984451584,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.299,1975-03-05,false,65212251.8303,2:9:34.28 +1656816018,20909111423164960,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.300,1979-05-25,false,954388489.642,12:44:14.9 +405917202,11932134209817282,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.300,2007-10-09,true,1213990276.03,13:29:30.12 +1143969229,9029733288608216,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.300,1994-04-23,true,1226601662.37,8:14:3.14 +1589836636,73429016008830992,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.300,1974-11-07,true,849284697.672,18:55:26.26 +-775992368,21442778138371832,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.300,1988-05-14,false,470763463.128,7:25:37.53 +863255554,16146448494878188,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.300,1985-07-05,false,74956170.5176,22:5:9.6 +1666636589,58737186982238960,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.300,1972-02-09,true,1241726610.86,19:43:44.12 +-121361169,22532408491762884,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.300,1968-02-17,true,171237054.573,2:6:58.49 +-2084467570,22779920932319908,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.301,1978-08-24,true,1200258391.49,21:1:49.34 +-1961487533,60689260939614608,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.301,1974-04-13,false,192358577.762,11:31:8.9 +-1091630352,66872595847692872,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.301,2011-01-23,true,832395383.359,5:45:20.29 +-1377017689,9490399381798072,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.301,1992-09-24,false,567629706.515,17:9:27.42 +-2083626003,40195196813472288,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.301,1956-10-13,false,1023206517.53,22:38:2.29 +-1386239300,17220948654288968,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.301,1962-06-10,false,1341221917.93,7:26:17.12 +1273330331,78888929211440336,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.301,1980-03-07,true,1134110692.61,19:45:9.3 +826003793,12422604947497258,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.302,1954-02-23,true,1055382241.74,14:21:15.36 +161284734,6677715331038218,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.302,2007-11-23,false,548561838.924,22:35:55.9 +194907521,25969331708434912,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.302,1996-11-22,true,1312025236.64,19:7:58.48 +-87667444,57904809339873200,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.302,1988-10-18,true,800361779.228,11:2:49.48 +1244314667,22154804737705300,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.302,1968-09-10,true,1392495883.0,9:48:4.37 +-684732814,83823043860614624,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.302,1960-08-19,false,623338816.597,5:28:48.38 +2078874407,13619017107539498,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.302,1970-04-15,false,1047212947.55,18:40:13.47 +-13170145,6045580604602307,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.303,1960-02-10,false,96600174.7839,5:10:2.9 +-1878912020,19478925615855648,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.303,1957-02-02,false,1165938637.84,20:40:31.25 +-1020162269,13386147570189732,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.303,1953-03-23,false,803877567.663,6:25:19.55 +877486545,64610667768572080,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.303,1954-09-01,true,639145925.458,10:22:48.21 +-1878105299,19757965202826628,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.303,1988-07-11,false,133411014.706,2:21:32.41 +842588131,30582304507339348,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.303,2012-10-04,true,777961373.252,2:5:44.12 +-1213807452,35434522128701236,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.303,2001-07-13,false,1152954026.9,3:20:16.21 +-2031119903,6585016636019671,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.303,1965-10-06,false,104615882.24,6:7:41.39 +2050389527,3649114446697267,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.304,1968-04-16,true,413935172.742,15:54:16.40 +1691082738,43883789781182800,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.304,1972-06-17,true,394375539.849,4:28:47.16 +2093165709,63418567193445200,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.304,2008-11-08,true,989188081.442,9:37:31.28 +-932679435,35481770877037712,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.304,1973-03-05,false,988366702.496,8:12:24.46 +-1116990058,76633394172736352,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.304,1951-11-20,true,1045954132.25,3:34:12.20 +-340095302,41749710598310000,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.304,1972-03-26,true,938296509.579,20:48:19.23 +1031994755,70772699210988936,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.304,1952-09-08,true,587322464.188,17:15:26.54 +1751618190,14553262272802878,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.305,1989-07-08,false,109445211.63,18:38:43.54 +316625262,64568262287064120,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.305,1994-02-11,false,1213648346.55,4:42:1.35 +-1719933403,31427194692939060,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.305,1969-06-09,true,905641006.905,2:4:15.48 +-1622176270,34563820335228088,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.305,1969-01-23,false,246711384.721,19:57:41.16 +-1888604715,85389507608893984,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.305,1995-03-26,false,5127871.95787,8:16:1.11 +-765019504,5418427312201605,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.305,1995-04-15,true,1209376452.44,14:14:15.48 +76393778,35611791729438436,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.305,2001-06-20,false,300200900.397,7:28:57.50 +-316667194,37864987745591552,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.306,1986-04-16,true,628357237.593,15:39:53.26 +-331644180,4758931397931244,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.306,2012-06-06,false,1238297167.17,21:30:55.11 +2063097787,71710258604575456,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.306,1965-03-05,true,586504956.322,17:6:22.1 +1292291764,62275133841643216,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.306,1996-11-20,true,288776264.314,10:10:14.1 +1653222674,61444340821081824,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.306,2010-04-21,false,999951721.999,17:19:15.32 +1453986123,29600377442416764,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.306,2007-06-27,false,1104746721.18,16:9:45.37 +1402288607,68353822691101536,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.306,1986-04-26,true,1195153371.47,3:35:7.15 +1401233001,7691894596857887,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.307,2013-06-02,false,57968184.2748,11:12:51.24 +-1629266744,61010276465520576,AK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.307,1968-01-05,false,862364987.7,19:55:41.38 +227477591,44578914169728704,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.307,2010-03-13,false,726971816.499,4:53:19.45 +1398741799,28492879139703544,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.307,2006-04-16,true,1179065802.02,1:37:6.40 +-305550301,66740842421629872,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.307,1959-01-23,false,1132494752.71,4:50:30.47 +-621910476,38824744506009160,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.307,2005-04-06,true,427221395.095,21:27:22.20 +124865425,41751825475784056,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.307,1954-10-23,true,201553543.486,22:32:17.12 +-64701410,89731582423183376,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.308,1960-06-25,true,368419746.392,6:4:47.35 +1370599271,3719037836919972,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.308,1954-08-27,false,579329408.901,5:49:29.7 +-173743200,74870846962192000,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.308,2003-02-03,true,318514967.081,16:25:30.27 +1851893853,66730907725813496,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.308,2011-10-19,true,1418542068.39,20:1:53.2 +-371911822,46457812577994096,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.308,1985-01-20,false,573907404.002,11:43:55.43 +-259407830,58986108789643264,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.308,1952-04-24,false,564913285.884,7:17:23.50 +-1551958529,30810315823688008,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.308,2005-03-13,false,432286040.739,10:56:19.21 +1413021713,2108737327435284,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.309,1967-03-06,true,56592908.4273,22:27:4.2 +-1263457565,15509930964213300,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.309,1953-11-09,true,1222396746.25,17:34:57.31 +-1007060944,47487184863684232,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.309,1990-04-13,true,749982066.977,6:46:36.56 +272754904,72121425930221856,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.309,1997-11-25,true,183862165.426,22:16:46.48 +1278453022,45826048288614864,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.309,2003-07-23,true,503537075.213,9:42:39.20 +489964459,2205180305959065,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.309,2009-06-27,false,90688319.3343,1:43:8.43 +-1554369440,83569016336194864,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.309,1986-03-25,true,1178776680.24,3:39:55.35 +-1189283811,4146378196614461,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.310,1953-10-13,true,1083636892.46,3:9:8.44 +2054369059,92203593218662368,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.310,1987-09-08,true,468516616.168,7:45:43.36 +-1101110042,22083298938303200,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.310,1965-05-07,true,351696534.115,7:56:34.35 +-477728169,1234469359809822,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.310,2000-08-12,true,1364003268.47,7:40:27.36 +-1202548410,71560517729611776,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.310,1965-10-24,true,85463081.7991,6:45:26.10 +255441665,61656030289871504,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.310,2003-11-03,true,1318347815.31,5:48:41.55 +1149402561,85203908934306752,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.310,1965-04-07,false,511270916.077,6:47:35.32 +-1887630985,31697628644105728,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.310,2001-09-13,true,840588519.929,20:32:49.7 +-1550268548,29129237731125596,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.311,2004-07-15,false,1201097479.5,18:22:12.40 +-695919352,13743463699554192,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.311,1988-08-21,true,398188314.404,5:52:20.33 +1362574222,71802953774915968,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.311,2005-07-10,true,1430797754.89,9:13:31.27 +-497360707,67854045219754168,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.311,1990-11-21,true,907679625.642,5:49:27.33 +253109868,62818583233832560,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.311,1960-11-15,false,176819497.758,15:51:15.13 +-207600984,24066550509668436,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.311,1976-09-16,false,812777549.666,13:12:15.20 +108097049,90133077428139968,AL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.311,2010-08-06,true,1075592300.02,18:18:32.53 +-869402250,85706582045279216,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.312,2012-04-16,false,1135032869.91,18:30:16.44 +296490823,39390936385889248,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.312,2003-10-16,false,107393815.435,19:47:49.6 +-2087181876,29028772946851484,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.312,1959-02-24,true,1077243971.9,17:35:8.29 +-1562138728,22542288455267420,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.312,1989-05-27,true,11442526.0464,6:52:33.16 +-1412752743,62528968797106672,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.312,1982-03-11,true,208519703.061,4:53:46.16 +-1334160798,38804107121913920,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.312,2010-07-01,true,376440955.334,17:28:20.23 +845014049,46073108164203544,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.312,1951-05-01,false,77973495.5367,1:50:26.26 +-329633849,510909754908026,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.312,1950-04-21,true,1296797801.92,17:30:2.6 +936099903,10666437031598418,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.313,1993-02-17,true,707345401.693,9:19:36.48 +1371125013,68845437300976024,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.313,1981-06-03,false,731950629.386,20:2:9.52 +376711703,42507549167077160,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.313,1990-09-08,false,1422531410.12,13:36:6.50 +975911043,30182732915568568,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.313,1958-08-21,false,727008794.282,7:36:4.27 +903206546,86832500732520016,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.313,1998-05-08,false,1265246561.02,12:49:38.19 +-1125727615,51600371794204408,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.313,1966-01-24,false,494692865.756,15:49:43.7 +-1505959345,64909523389946688,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.313,1973-01-26,true,234214526.221,21:48:14.17 +-703261018,65006465541888296,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.313,1950-04-25,false,641825505.017,3:5:6.19 +1607542851,81058713852021488,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.314,1977-01-13,false,220976491.795,15:12:42.49 +1391745863,68736347813698568,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.314,1999-07-02,false,1400906383.52,21:3:27.29 +-1471899330,67040327319634896,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.314,1993-01-25,true,794259509.956,12:39:30.49 +-1244417495,29892420964160788,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.314,1998-11-05,true,481473850.438,16:36:21.6 +-574142269,48013005537775944,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.314,1972-01-18,true,52565017.9419,13:31:21.46 +-393774124,29979642979204804,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.314,2007-05-23,false,530991193.602,18:32:36.9 +988042446,52651513922615736,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.314,1979-06-06,false,840587461.357,5:48:34.55 +1922676713,78211573126306064,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.315,1981-06-18,true,1223600813.89,7:30:53.36 +809670424,75672452287746384,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.315,1960-09-22,false,964097612.46,5:17:51.23 +760430086,77898595212693808,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.315,2008-10-11,true,995281667.574,12:50:27.35 +-155578896,24366141868455864,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.315,1998-03-08,true,884130866.587,21:40:42.54 +983996437,57200984508169144,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.315,1968-09-23,true,963615916.849,20:23:37.32 +1304455930,62760769250664016,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.315,1995-06-09,true,722994417.63,1:38:20.12 +181882990,16615116514110402,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.315,1996-06-17,true,276440207.853,3:58:6.57 +-1959977723,79099808230479264,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.315,1952-11-15,true,1243400190.77,19:56:5.41 +-34002029,52775572403911032,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.316,1974-09-01,false,596974291.969,9:44:12.9 +-462113294,38158783589287720,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.316,1967-08-18,false,41345554.4624,14:27:16.31 +78433303,21179967103681876,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.316,1972-03-13,true,299429843.631,12:26:38.12 +1054692887,48594837806934192,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.316,1958-10-09,false,139973433.738,18:50:6.36 +-401456423,38857455670157880,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.316,1990-08-14,false,817932128.21,4:39:42.27 +-1092116454,80766989034061296,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.316,1998-08-23,false,600403164.208,6:41:19.40 +-1645872784,42091129153264864,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.316,1988-01-05,false,138548917.246,11:21:4.33 +554209445,77872670043099328,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.317,2009-04-11,true,1392235911.7,15:1:15.32 +-381237435,12978216240861470,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.317,2008-03-17,false,1254019278.49,22:40:31.4 +-2051663318,81438926216429200,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.317,1962-02-11,true,774392837.18,7:12:42.27 +1576109045,72301095678520640,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.317,1986-10-16,false,919389374.73,15:49:17.31 +1257866545,48885695538753568,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.317,2011-05-24,true,775042419.306,4:13:1.35 +398889723,28955421449122376,MT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.317,1999-03-26,true,74787977.5684,7:11:1.2 +368432813,31024522575050700,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.317,1991-08-26,true,972808559.465,17:17:23.19 +941197167,31963109853996904,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.317,2000-08-19,false,643281450.399,21:51:35.14 +-835288746,267922684373258,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.318,2005-01-05,false,865028833.667,18:41:11.56 +-1213626479,4076200776235356,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.318,1952-11-25,true,644934401.842,20:15:17.25 +-1856572175,24877716957219512,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.318,2007-01-06,false,239760867.089,19:53:30.52 +489087256,74871042263298880,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.318,2010-04-09,false,534563438.936,10:57:50.37 +1028929798,36550434798223872,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.318,1969-02-11,false,637244008.415,18:36:1.44 +1701002597,78184869717548368,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.318,1953-08-25,true,451618077.293,12:10:29.51 +-1938393293,67722000942608008,OK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.318,1978-07-05,false,842659804.571,13:26:13.29 +-1084261897,42840079290693640,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.318,1988-07-04,false,505079273.052,22:5:26.52 +588950062,17250043458049116,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.319,2013-04-16,true,287668039.05,1:40:40.35 +881817588,3585595916261683,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.319,2007-02-08,false,1193413665.66,11:55:21.24 +1474771679,23427511222898536,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.319,1982-05-03,false,718405526.817,4:5:46.30 +492927700,60997230293620736,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.319,1956-02-06,true,86337531.1633,2:28:15.46 +93895661,91280686006085968,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.319,1986-04-05,true,1214957210.06,21:51:36.51 +-142328277,23412204558802352,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.319,1950-08-19,true,937366782.039,20:28:5.2 +988078892,68771607735018392,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.319,1983-11-04,true,1182360573.18,14:40:29.57 +1933127159,41208662210923064,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.320,1986-04-25,true,832210720.255,19:28:24.18 +2097906442,86435945183019120,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.320,1990-10-11,false,244806280.158,20:56:14.25 +516555895,68233692588939408,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.320,2000-02-08,false,131066180.805,13:9:45.50 +2099933946,64361349926459376,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.320,2013-11-14,true,190058833.616,7:8:18.9 +1637963360,55234210936373120,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.320,1973-02-09,false,346096536.913,8:47:13.42 +-2101270128,36829994257514232,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.320,2007-10-26,false,710881691.557,17:34:56.1 +-2068257443,5928929439659827,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.320,1988-03-07,false,211020658.258,5:43:19.48 +-573759162,20207858764660380,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.321,2003-09-09,false,703865196.587,19:27:42.20 +-17391516,49203361343750240,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.321,2012-02-07,true,1427645534.72,17:42:30.58 +2103802780,78887789850345440,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.321,2012-07-23,false,1022044607.46,22:33:17.41 +1432764757,61979826588242752,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.321,1971-02-12,true,990617.644707,11:58:28.33 +649123344,91803793108735088,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.321,1994-10-15,false,728888771.46,10:30:54.46 +1878879850,40415011020562912,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.321,2003-03-27,false,1032858555.52,21:31:37.30 +389781178,6524828574573169,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.321,2008-02-08,true,1297895220.9,17:32:29.42 +275290308,191792028522547,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.322,1975-11-18,true,169472266.825,19:38:7.10 +-77113515,75529722262167536,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.322,1975-02-14,true,483935850.917,10:10:13.44 +819487611,3206820616752056,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.322,2011-03-25,false,408815201.377,1:26:33.13 +1557361393,4248333108083272,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.322,1962-04-25,true,712602937.701,6:34:9.27 +528790533,89493288416062480,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.322,1974-03-07,false,715282744.476,10:17:4.10 +1195614956,83085880896789152,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.322,1952-02-06,false,1335578631.84,16:20:49.11 +1026129970,72281447485118800,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.322,1994-07-06,true,482638438.127,9:17:33.48 +-1401961998,61060514228485520,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.322,2012-04-22,false,405959811.982,15:15:39.51 +-1333788041,67167062774193328,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.323,2001-08-18,false,206716856.07,6:51:13.27 +794077193,51998647300504752,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.323,1998-10-22,false,915666192.593,9:47:31.5 +78242037,66004210153391696,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.323,1972-01-24,false,460292162.924,6:27:43.30 +2064055995,47978257940302200,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.323,1987-10-25,true,565316930.222,4:3:49.3 +1972904353,78300067344942192,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.323,1996-02-01,false,666579437.214,1:57:17.35 +1369350597,46256038147219968,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.323,1986-09-06,true,462769394.666,21:34:47.15 +-481184281,81047359163249808,OR,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.323,1993-07-18,true,520827339.828,4:57:18.32 +548700567,51397739112536104,TX,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.323,1996-06-11,false,683703800.345,1:28:36.38 +-662609086,54538867188316664,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.324,2013-08-27,false,221223726.384,12:24:52.55 +867928541,70788003193907040,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.324,2013-02-04,true,774634082.494,10:40:10.55 +2010779531,18637280547300496,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.324,1955-11-22,false,180690152.76,6:14:22.34 +713306757,69155842630139968,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.324,1989-11-20,true,743196401.616,18:10:28.5 +1372227540,16587401949957898,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.324,1981-04-09,true,1420020185.32,17:44:27.29 +1739042208,37426842885037088,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.324,1964-04-03,true,426970591.882,1:16:4.56 +575128307,54450582367641272,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.324,1982-11-07,false,1299237871.12,20:50:49.40 +452259738,65084147438219632,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.324,1983-07-25,false,1024489125.46,16:23:20.52 +708901890,39414282019634912,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.325,2012-08-11,true,785420794.038,11:49:31.29 +-1256060203,23437740237535920,MI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.325,1972-02-13,false,856003328.097,19:30:16.46 +1051841791,60803505588572368,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.325,1987-02-22,false,1152891653.25,20:17:21.44 +2091774798,33469287862792848,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.325,2000-08-09,false,1306227608.07,22:4:40.26 +1542459203,83769004404962784,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.325,2006-06-11,false,533207186.612,9:23:49.44 +-844953511,26039360824456428,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.325,2001-03-22,false,1295825144.54,18:34:57.56 +-308115399,26148331955988796,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.325,2009-05-21,false,26466097.8037,6:57:29.35 +47045812,78569321938260304,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.326,1993-02-07,true,1401462554.64,3:58:9.6 +390524244,45739713016789648,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.326,1956-10-02,true,696112070.744,21:50:22.51 +48064731,66207131491598416,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.326,2012-10-05,false,752334421.138,16:47:8.10 +1816021252,49267453352851120,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.326,1989-10-10,false,1059370066.22,10:37:36.33 +-2025830366,69258617527263200,IN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.326,1989-04-08,true,437108490.805,12:23:1.33 +-743908284,17004916271608444,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.326,1993-06-04,true,1287405876.24,20:54:17.57 +-1135211829,34725360967580744,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.326,1986-04-05,true,425246720.003,19:49:3.36 +-417302727,10213546225520774,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.326,2013-02-26,false,1359677354.28,12:31:23.18 +1325626775,19201719639221524,AL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.327,1955-09-02,false,1057714457.29,3:40:32.31 +1604918687,43722225280169176,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.327,2000-05-06,true,96342395.6672,12:1:17.27 +-330047872,46264491357621864,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.327,1960-09-11,true,626311805.4,2:8:22.13 +-1778356728,72336522057058176,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.327,1991-11-22,true,729085323.986,10:22:55.21 +-1407114331,26544756344784908,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.327,1961-04-09,false,784570047.071,20:48:52.26 +-1684629739,85288863238365728,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.327,1990-03-24,true,876150905.568,4:39:14.58 +897100432,62410845467554232,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.327,1988-07-07,true,1024947986.07,17:29:19.8 +-1926884937,20548954020442452,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.327,1969-04-17,false,742063551.06,19:42:12.10 +-1204588051,60811679527451824,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.328,1991-09-16,false,291928040.045,18:42:35.5 +-42736796,22241576776783032,SD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.328,1987-04-15,true,701537931.06,2:13:38.5 +-1347667579,8130177596672297,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.328,1971-06-03,false,511481236.311,10:36:22.33 +-1435191938,31414854842533336,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.328,1973-11-03,true,991947843.496,17:15:1.22 +1621244821,2460692230153584,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.328,2000-08-08,true,622245696.181,17:56:7.33 +-1527874092,69535811841247880,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.328,2007-08-13,true,56693823.0441,5:12:48.43 +1671213371,73897769913267616,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.328,1979-02-18,true,858533338.141,9:3:25.3 +-1368011783,34614054642526444,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.329,1953-08-13,true,1405439231.7,3:39:10.41 +-640963332,25164422091878740,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.329,1967-04-09,true,471887817.046,12:19:31.56 +1900140314,65349823006425528,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.329,2012-03-12,true,128521809.01,7:2:48.16 +614349994,60935057946206128,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.329,1998-11-23,false,788018810.862,8:26:51.31 +-1625390282,15626480909261680,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.329,1994-01-26,true,307973135.437,20:24:58.20 +1291037028,9044225586175448,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.329,1956-06-05,true,1160461536.51,2:31:18.3 +191396980,19657362973598456,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.329,1970-03-17,true,23043764.0048,5:24:24.58 +609042389,44483181951680520,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.330,1962-08-07,true,126963569.697,15:28:51.5 +1665142028,55495896428211968,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.330,1986-11-11,false,571939490.096,11:13:52.11 +362500862,31951522035787828,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.330,1959-01-16,false,1221757890.21,1:23:26.46 +-1866148113,91584788645607392,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.330,2011-11-09,true,1388215161.7,14:29:6.40 +-1664891439,12084733020869960,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.330,1968-08-01,false,1307919840.2,10:30:35.58 +884010237,7057958382344233,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.330,1953-02-17,false,1090736428.53,2:36:41.4 +-861576521,85674137857632800,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.330,2007-03-11,true,533177214.572,21:34:38.39 +944101604,56544931605191920,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.331,1952-09-25,false,120183171.543,19:37:15.36 +1932260777,74670573071126896,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.331,1998-07-15,true,1027238258.66,13:52:44.50 +-1636785931,1295515346929889,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.331,1956-02-24,true,138944259.062,1:29:41.14 +-1234576309,3971085671243049,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.331,1972-04-27,true,282090136.175,20:58:49.4 +428402038,79304037098161008,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.331,2005-03-16,true,1321254514.76,3:9:19.28 +-1965695662,42717860833260440,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.331,1981-09-03,false,1026028460.02,4:32:18.10 +-469134584,33456692907661036,MA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.331,1978-03-17,true,449036455.231,3:46:38.6 +-574232884,47501845754716552,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.332,1990-10-13,true,545723484.89,22:47:1.22 +-1848401263,72649863868526640,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.332,1951-10-25,true,1023793056.46,21:18:2.13 +-537668561,31577601426968916,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.332,1995-04-05,true,105789682.056,14:19:44.55 +-448615011,1013533089933445,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.332,2006-05-03,false,1280158939.82,5:23:29.34 +-721023817,2816689499150643,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.332,1964-11-09,true,1065395725.05,8:20:22.34 +1862412994,86828289282645808,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.332,2003-08-16,true,227692119.142,10:34:5.49 +1317264349,51580411359367800,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.332,1980-08-16,false,1053525543.99,8:15:39.46 +-32332999,42057190711670368,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.333,1983-08-16,true,173224322.456,4:10:22.34 +554455457,29700674064745432,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.333,1959-01-08,true,1148522458.14,22:42:1.36 +-1733563343,10785799040297288,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.333,1951-05-04,false,1002551063.67,15:56:9.29 +274909055,48748407775246416,MI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.333,2009-10-25,true,676128958.965,18:19:11.5 +631965489,71360429979241312,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.333,1974-01-11,false,1371812797.94,20:41:14.50 +-24253970,9791681735084186,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.333,1967-08-20,false,868437032.634,4:44:4.1 +-1811210507,88588992294279728,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.333,1953-10-15,false,1323149120.22,2:38:4.33 +-417018999,53241952435041144,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.334,1971-08-15,false,1346426945.36,7:7:45.34 +-2095970409,61910083298831184,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.334,1997-08-07,false,292723198.917,5:42:4.5 +2140422362,61120227047289672,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.334,1970-04-18,false,1063445384.04,17:10:54.52 +-360726313,48822887318545328,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.334,1982-09-07,false,1394058441.4,14:23:14.7 +779314534,62946457192517072,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.334,1979-05-05,true,798341048.992,3:56:30.6 +-1606776697,25098491503370968,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.334,1996-03-17,false,392137800.156,2:2:43.50 +1801264575,37478907264135288,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.334,1999-04-06,true,1252732267.35,2:20:11.10 +-1854749993,62815046498760360,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.335,1986-04-25,false,207264116.194,5:46:46.26 +1849541307,64422800566843968,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.335,1983-05-23,true,1088546381.39,9:5:4.7 +1418127031,75717866735182944,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.335,1974-04-08,false,1232370692.8,19:2:7.52 +2023742298,46660505352887176,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.335,1951-08-04,false,3109656.3139,14:43:11.35 +-193571849,3977270930537492,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.335,2005-03-22,false,1364182539.7,18:30:18.22 +-2065946994,237369672038195,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.335,1968-04-15,true,1417961739.3,21:34:42.41 +837913915,24500072230020036,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.335,1967-03-12,false,730533821.899,19:18:7.35 +1909020085,85098548196938400,CA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.336,2002-01-09,true,653123649.092,5:42:19.3 +-1670558225,81969117375033328,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.336,1970-01-01,false,641133943.018,10:15:36.8 +-1907368514,3433001099840409,SC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.336,1975-03-08,true,615934154.469,1:54:2.42 +-480123574,5938539344775414,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.336,1977-05-22,false,264792914.118,22:7:49.4 +55881718,54774607147723232,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.336,1958-06-22,true,237927137.41,17:15:39.1 +-1697257126,34691123293644288,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.336,1980-05-12,false,1235585878.56,1:23:56.56 +-2064512955,66654882857632808,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.336,1999-03-07,false,478583167.143,5:2:45.25 +-1697742453,73683612811155552,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.337,1951-09-27,false,1025492029.51,4:17:43.38 +922786500,58404756153137336,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.337,2009-05-06,false,136801565.267,14:39:24.6 +-97974361,58086938223965736,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.337,1994-03-07,false,115793197.006,19:12:12.27 +2084655647,50653036909134360,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.337,1993-01-27,true,851265090.285,3:38:32.24 +1570662590,62288930192296488,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.337,2013-04-26,false,635957334.984,2:13:14.47 +-812410169,60757433022833848,OK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.337,2013-04-26,false,1121571764.18,4:51:54.22 +916493567,34564453338447260,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.337,1969-08-04,true,294313520.113,12:13:11.29 +1284928716,30812490574039092,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.338,1991-06-19,true,56511553.0355,6:4:53.58 +25641147,61982736522337320,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.338,1964-01-17,false,1231232140.73,13:54:42.46 +1977502142,60890625604095824,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.338,2009-05-06,false,257525524.938,3:13:9.43 +-1804533061,80075484782715632,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.338,1971-10-09,true,76870786.1197,3:13:9.30 +-1728969384,30742882971067392,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.338,1951-01-23,false,306515056.899,4:56:38.34 +1022141520,6556634515102956,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.338,1992-01-16,true,943893179.824,16:32:27.42 +1782250795,52750279463288304,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.338,1972-10-15,true,83328394.3105,19:5:31.39 +1452521802,5315241776178094,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.339,1958-01-08,false,298393683.467,11:49:41.26 +-32067905,74112964027385424,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.339,1959-06-22,false,847142600.001,17:17:4.7 +-1354507492,57832326188102808,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.339,1976-02-08,false,319022978.983,3:23:40.19 +-1595300651,89816580481012848,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.339,2001-09-24,true,143331518.73,12:46:58.5 +-290005128,81196345832716656,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.339,1974-01-03,false,373872380.937,9:42:40.45 +-135474963,16585805427116810,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.339,2008-06-01,true,776306020.187,6:58:11.15 +692982708,16271646878342104,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.339,1979-04-02,true,1143512035.84,1:14:33.2 +1442637792,12369050740768594,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.340,1969-05-09,true,988403279.014,11:49:42.46 +1746337734,9788152701057486,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.340,1961-06-19,true,546296788.357,16:13:12.52 +-1451296826,624648821979914,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.340,1961-03-07,false,647081310.768,16:53:53.57 +-2075383610,42963579665268136,MS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.340,1983-01-08,false,1145353861.86,6:33:38.32 +-658666281,35074979232897240,CO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.340,1967-06-01,true,856158877.342,22:28:3.1 +282234453,61421548919512248,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.340,2009-11-05,true,1061001097.61,19:56:51.16 +-793396480,18077312085844500,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.340,1957-09-19,false,1249443537.47,3:1:46.21 +-993404370,66805654669156232,NJ,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.341,1967-04-17,true,489071566.434,6:39:33.9 +680967134,44250105254188144,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.341,1962-11-18,false,1162323420.59,12:54:36.13 +1791100972,56483661443946136,MA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.341,1990-11-18,true,83645857.5016,9:25:22.52 +-60763760,16740966093973166,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.341,1984-08-06,true,1354347446.14,1:3:24.50 +-1183819792,83568463810375344,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.341,1970-11-25,false,439488985.343,12:38:52.34 +699312616,19493683873609656,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.341,1996-08-02,false,586609854.462,19:9:19.40 +-289253328,6855962688031816,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.341,1992-04-25,false,39505282.7184,20:23:49.53 +1489308687,70389013627777752,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.342,2010-04-06,false,723343858.861,13:43:56.3 +2059182041,41596203476906392,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.342,2002-11-09,false,312803585.115,2:53:32.54 +-787514909,34531536436048168,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.342,1960-04-03,true,1153743638.21,21:35:21.41 +-483598551,79008565669822896,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.342,1958-04-16,true,461421576.339,17:36:55.12 +64902047,20540634873681028,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.342,1999-02-15,false,950065321.872,21:42:29.44 +-1078387651,4745834342890506,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.342,1971-08-02,true,517825991.027,5:9:1.1 +142883141,6749165324994038,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.342,1990-03-18,false,1101639591.86,5:28:33.16 +-665183637,44143150645352840,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.343,2005-08-23,true,1252270650.06,6:49:37.57 +256232429,58886331852169688,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.343,1976-06-07,true,140341826.731,1:46:25.12 +-2060980508,30586855775187620,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.343,1997-03-26,true,401061313.989,4:24:51.31 +880890035,89142034775604352,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.343,1983-07-19,false,826934256.754,2:43:51.56 +-1691724584,18971644557973052,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.343,1980-10-16,true,1374831216.02,21:21:27.10 +1204874252,8088858667044639,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.343,1994-01-06,false,1107295783.38,12:26:50.36 +889021421,90712577313452528,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.343,1953-01-10,false,1125324759.2,22:28:35.9 +908501765,36856272463838944,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.343,1997-06-09,true,49690831.0277,11:45:23.36 +545042988,80618754420141424,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.344,1958-11-21,true,1156072824.87,9:25:17.2 +1132634173,52381207716478360,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.344,1964-11-14,false,793576178.996,1:14:44.19 +1607402470,36482946163223912,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.344,1951-02-22,false,26940185.4047,2:46:5.22 +1863366981,7299783190914468,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.344,1952-01-07,true,757260303.913,11:1:52.35 +413923810,58921333868396456,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.344,1993-06-10,false,382676088.729,21:21:5.55 +-1414739599,18514899619035924,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.344,1974-06-25,true,328564934.632,20:1:45.46 +-1742037234,52597340142063544,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.344,2012-01-12,true,11678469.9803,17:7:47.19 +197408910,66391975083404912,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.345,1972-11-10,true,653441719.364,17:18:17.30 +33237704,44147026134807928,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.345,1993-01-23,true,742639681.812,5:27:49.27 +-972802326,51746339282127184,IL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.345,1984-10-27,false,731249557.305,7:46:33.36 +-1395214919,25732560766344796,MN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.345,1994-02-09,true,912860189.294,15:51:43.54 +-1860345377,63047529615607184,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.345,2003-06-08,true,307920843.224,19:58:47.20 +631236586,77069241381244944,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.345,1951-04-09,true,7346758.56185,15:47:10.54 +-1847342217,53529096081362640,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.345,1982-08-03,false,646733034.025,9:41:46.35 +1823404310,76516048976540240,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.346,1997-10-10,true,12021271.3896,9:15:6.21 +-619690107,52105419265200792,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.346,2013-07-05,false,193586386.701,16:38:45.34 +-735539431,15287737912898766,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.346,1994-02-25,false,946703896.244,8:37:13.41 +1488628847,8933274625153567,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.346,1957-11-23,false,1333029028.14,17:12:30.16 +1176512621,17303627163590370,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.346,1966-03-08,true,1015405217.41,12:16:45.12 +-1379632772,90497150109321808,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.346,1968-11-27,false,1402137552.87,21:19:49.16 +-25927667,70879352719972640,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.346,2011-01-27,false,575180767.447,18:11:11.10 +495312718,12514122317429924,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.347,1995-02-19,false,758280985.342,22:23:35.48 +458525681,57193071278783568,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.347,1979-03-06,false,700023553.836,7:2:29.49 +-268771618,7130626298827223,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.347,2011-10-08,true,1209960796.88,21:15:30.44 +-934264102,80023823310773120,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.347,1981-01-08,true,74637150.2831,10:9:50.38 +-1988049545,51100276840900872,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.347,2002-01-24,false,989054561.488,19:16:50.49 +-55276616,15927772665263524,VA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.347,1978-09-15,false,450177351.775,7:51:7.49 +803333658,51701329710757552,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.347,1957-02-06,false,1380661420.21,6:21:3.37 +-543590376,88800253377970016,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.348,1983-03-12,true,1273862745.83,16:44:30.49 +-528557071,37569635618244392,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.348,2007-03-05,false,569774549.302,7:58:8.27 +1435346738,53450265539336072,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.348,1994-03-22,false,16851830.0734,6:37:18.8 +524325091,12292645595414466,CO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.348,2006-11-26,true,1309106410.7,6:27:41.52 +583461709,40343194404228496,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.348,1968-08-04,false,1159518415.06,16:2:53.48 +-1937586898,38947714867000496,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.348,1974-01-20,true,1340958405.59,6:42:44.27 +-378792098,34395203747598552,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.348,1976-08-15,true,1366343992.51,11:56:48.10 +-1602374984,31698470739437024,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.348,1986-05-15,true,811075855.546,2:15:14.46 +-1830548997,65234081592217664,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.349,1994-03-14,true,294667608.119,13:2:48.27 +1752356378,71544254884372536,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.349,1956-07-11,false,984822228.384,9:41:47.22 +-2131129241,18819433608882504,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.349,1968-09-09,false,726261477.63,5:1:29.18 +-1319373956,77866442728852240,VA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.349,2011-05-17,false,850711811.155,8:21:51.47 +385707187,18762328635982868,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.349,1952-07-03,false,983307132.702,21:28:38.27 +2046655754,2370164156895099,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.349,2010-10-10,true,1392203159.28,22:14:43.26 +-2123547648,50659932509291176,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.349,1971-04-19,false,363782057.169,17:31:21.51 +-1112189072,26721868982120900,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.350,1976-06-16,false,39448031.9552,12:36:20.29 +-1914520532,23984199000150652,SD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.350,1997-06-19,true,960674885.498,10:24:38.3 +-1306276225,21699450743072964,MS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.350,1987-04-24,true,994395098.665,16:8:34.4 +464208929,26396772389585564,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.350,1993-09-15,true,777701668.463,9:54:24.36 +-910823203,75523732370231952,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.350,1992-10-12,false,444238902.416,1:49:56.8 +1179876675,25003021394621084,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.350,1950-04-06,false,923467500.508,9:50:11.39 +1112100685,55116317725535560,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.350,1955-07-21,true,591865183.66,12:42:29.54 +365204546,91866106448905120,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.351,1979-10-27,true,629291506.216,2:43:27.9 +-1808404494,18572429393932184,ND,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.351,2011-05-14,true,947230820.673,6:19:3.42 +1560355954,58724114194990416,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.351,2003-06-26,true,792267616.703,2:36:27.36 +-2032857332,257861409169172,KS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.351,2005-09-17,true,714326492.387,16:47:36.21 +-1882599543,41627997115087296,NC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.351,1993-07-15,true,731551674.142,6:55:24.23 +1689763397,13536798080098428,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.351,1952-11-27,false,388707572.984,3:55:7.36 +-1198343663,28879762365091696,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.351,1972-03-26,false,413512069.897,8:6:22.50 +1621512077,68448723733382360,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.352,1996-02-22,true,630475723.25,5:10:50.38 +725133748,77284439414902304,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.352,1971-06-06,true,492082641.981,21:15:22.28 +-1958149333,13995650521499342,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.352,1968-07-22,true,1013260844.33,21:30:8.10 +1127190081,91212557112505600,CT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.352,2005-05-01,false,41511047.7589,4:53:23.57 +-717067892,43596935241275000,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.352,1974-11-20,true,192857184.249,14:7:57.19 +1730252152,1400374536491602,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.352,1984-03-27,false,1142202427.21,15:39:2.6 +-1028260982,43127380793182176,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.352,1976-03-26,false,38173500.3417,13:6:27.14 +1120451803,20396721223621296,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.353,1971-09-02,true,1182846090.11,17:25:7.13 +-1758081131,79282815873454016,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.353,1952-02-06,false,137231351.509,17:15:58.58 +-1916814561,52865308913069888,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.353,1985-01-01,false,1359464918.95,11:25:52.49 +-601461221,62654418386551736,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.353,2004-04-20,false,1356006744.75,5:38:25.20 +-1536539322,14104270242975622,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.353,1985-06-13,true,481416698.854,13:56:30.2 +1334151036,43954131986425912,AK,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.353,2010-10-26,false,855636420.207,4:2:56.15 +347259569,1154212213074114,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.353,1971-01-20,false,805903498.228,8:26:46.41 +1673523429,24074985131636308,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.354,1984-02-21,false,1129409645.39,4:37:34.20 +-638917476,63392397704378992,RI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.354,1957-08-07,true,1291875741.18,4:4:31.54 +21684284,63608521288544880,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.354,1952-01-27,false,922892633.052,10:11:7.21 +-1199411626,15000881673817682,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.354,1989-08-02,true,417066296.382,16:3:53.56 +-1157361292,65641556934936392,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.354,1973-06-16,false,1224527091.02,6:31:53.4 +-2039051939,79060307905830608,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.354,1970-03-17,true,970954200.607,6:9:8.34 +-1545052252,61550651421657408,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.354,1955-10-10,true,634904613.565,12:42:39.9 +1223255124,59163114064898632,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.354,2010-04-12,true,710137917.299,17:5:34.52 +1706648445,41336472779573880,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.355,1950-01-19,true,1199566260.03,1:30:31.2 +-817306357,39736718578062400,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.355,1957-03-10,true,1159937430.76,10:8:17.42 +-58569801,18452280552251700,IN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.355,1996-06-06,true,1249164929.01,11:19:37.32 +-1532049848,58072702090537256,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.355,1973-11-14,true,36386382.573,7:27:27.35 +772068675,49607959101179096,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.355,1958-06-07,false,349897825.334,19:56:9.20 +789814593,61923572748373320,NV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.355,1964-02-16,true,933197531.079,3:56:35.51 +453624426,76479116278570576,TX,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.356,1966-02-27,false,436067721.304,11:55:22.2 +-496361697,51631898807799440,FL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.356,1988-02-12,true,912444308.174,12:48:52.34 +-79040559,5883638965118751,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.356,2001-01-27,false,345732370.989,20:7:31.24 +2102207282,70549202698897872,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.356,2012-07-17,true,88776115.5191,3:15:48.54 +-550397310,59735507191331312,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.356,1990-10-18,true,266883787.742,20:46:40.4 +1937091060,73036618795886688,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.356,1950-06-02,true,722828492.935,7:54:19.58 +683697281,67458888569090728,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.356,1978-11-20,true,1122502130.87,12:54:37.53 +279043201,34680093423656512,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.356,1970-11-24,false,1411708781.56,5:25:40.44 +1926226161,61043651831900704,TN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.357,1990-05-16,false,932791670.971,21:25:7.39 +870453900,5188292823278531,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.357,1979-04-05,true,763078934.988,4:56:10.13 +579230413,33277668640930928,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.357,1957-01-03,true,940669653.76,21:35:55.51 +-2071990614,66203266808993664,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.357,1997-09-15,false,521839675.562,9:29:36.57 +-2064120471,72546614081179488,ID,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.357,2012-07-10,false,1144142224.82,21:51:43.7 +-672356236,425686976808376,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.357,1950-08-24,false,611619586.583,7:34:35.40 +-167513084,35728205503443456,NY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.357,1987-06-09,false,683698376.316,11:15:10.16 +1002856285,1007605491082864,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.358,1951-08-18,true,114002904.621,19:51:26.27 +-1922605929,14279859488713248,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.358,1965-09-08,false,690168295.066,15:41:26.58 +1827371015,6532395263054275,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.358,1950-01-08,false,785393023.636,2:52:46.32 +-1586915194,23801745902296104,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.358,2007-06-05,false,1153929217.43,8:57:58.38 +770901588,34741594433593160,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.358,2006-05-11,false,579832291.654,16:6:57.18 +1651121848,60656370481048328,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.358,1983-04-22,true,129829231.253,19:28:13.4 +-1615146478,1733917256349225,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.358,1980-07-20,true,372545339.486,16:19:4.25 +1849614444,14118312489682432,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.359,1969-01-24,false,343392509.898,19:46:20.9 +-2119102131,67612705184424640,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.359,1972-09-09,false,646785041.346,22:36:58.9 +-1035965739,34254731900515320,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.359,1994-11-08,true,505944701.973,17:55:22.50 +32201165,65779579186837944,WV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.359,1989-05-23,false,1147931280.93,9:52:2.35 +-1383213313,57372045756204784,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.359,2011-01-03,true,256201601.208,8:42:52.3 +1505825490,65446618575318400,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.359,1978-10-25,false,725244063.39,5:35:30.26 +-690453859,8903535497949891,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.359,1958-07-16,false,1217422852.12,7:46:21.11 +-551741678,18918654892453416,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.360,1981-06-01,true,112504639.392,3:15:18.53 +-284589404,15788889290576354,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.360,1980-04-21,true,1149871149.47,11:53:33.58 +-223288132,17090950323199078,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.360,2007-10-10,true,1368351921.22,7:22:58.55 +1004293653,52836056419157360,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.360,1974-04-06,true,1357083220.08,14:50:17.55 +1702946814,71410125040540232,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.360,1952-01-13,true,692806994.435,8:20:3.21 +795354156,1362420893599508,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.360,1976-10-20,true,255365451.573,13:43:20.54 +-425760059,8013269949939927,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.360,1979-05-13,true,667467846.01,6:4:27.4 +-765235339,91625305873753824,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.361,1958-11-19,true,197578020.737,16:5:44.9 +-724957205,49842444875226536,GA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.361,1997-10-07,true,733302011.019,11:47:15.29 +1745752549,77030438149632992,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.361,1953-10-04,true,1417425040.51,1:51:16.1 +253120062,13094373982386474,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.361,2010-05-09,false,22671690.9273,4:20:28.24 +-571203936,77086963922952400,SD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.361,1968-02-20,false,588940217.032,1:42:16.1 +-1710266039,84180141637737200,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.361,1992-04-24,true,1361910980.84,2:10:5.33 +1621897576,16638580224856454,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.362,1963-08-25,false,855211559.531,9:54:8.31 +-1445048016,9794320429435198,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.362,1970-11-09,false,184294438.952,13:46:31.14 +-429053562,56970347143781600,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.362,2010-04-02,true,370393929.563,5:6:7.41 +361872765,25449113395928884,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.362,1950-10-12,false,446701519.467,8:17:4.50 +1030929796,74110319950571568,NM,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.362,1961-01-09,true,99429169.9697,6:45:32.26 +-1282995842,54172566494164376,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.362,1972-07-21,true,564384196.714,19:10:19.51 +1837285526,40740313876503880,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.362,1982-09-19,false,1204326252.12,1:24:49.20 +-1031785210,57768520355439384,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.363,2003-10-15,true,398798437.0,12:8:32.30 +553928882,31666663287842940,MT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.363,1978-06-09,false,1358249147.6,21:9:6.37 +-520065324,64910027266025112,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.363,1973-01-22,false,1294102942.42,6:37:38.4 +-831684894,15192625565340796,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.363,1982-08-27,true,252171499.84,19:5:13.37 +-982821451,15900371867107072,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.363,1985-03-18,true,683728781.772,13:51:38.27 +1509188978,11497339358520852,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.363,1986-05-26,false,1328125196.49,8:9:34.46 +876284025,27877281860406784,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.363,1976-01-08,true,713188711.876,10:31:45.33 +-29478825,73603470672638656,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.364,1997-03-25,true,838923419.512,19:54:26.44 +-140609448,63977065523624624,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.364,1970-03-19,true,792047050.573,14:27:46.38 +-2046016583,84126275293022544,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.364,1976-04-18,true,1042296982.47,21:14:10.8 +-1293013046,24657925654105804,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.364,2000-05-25,true,379316718.906,3:9:14.47 +838597457,2622520465312389,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.364,1970-07-06,true,195825472.5,22:24:36.26 +-697458303,44495667933044840,WY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.364,1994-05-16,true,337319962.975,16:32:10.43 +1487274133,72396893349194800,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.364,1993-09-27,true,31785952.8603,15:22:4.21 +-2029512489,83514238253353424,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.364,2013-05-12,false,136983268.235,1:57:36.2 +-2090126404,39449407416558616,NY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.365,1962-01-02,false,534930293.742,6:18:1.10 +37221452,33078812666712168,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.365,1970-04-15,false,156713175.923,5:15:47.27 +597142379,33732333963906428,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.365,1964-10-08,true,184937208.776,5:31:19.12 +-163043080,81887145581628448,IA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.365,2011-11-25,true,1379411132.44,6:44:9.50 +-379077546,48082867011587808,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.365,1983-01-26,true,1218907557.41,11:29:10.47 +1878205874,68250880461388184,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.365,1976-04-08,false,834418880.698,14:29:54.56 +1596531259,65619384757870376,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.365,1965-07-09,false,993659113.495,15:8:23.37 +-213660068,82736765514826848,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.366,1956-11-01,false,1265274761.06,7:9:44.48 +1018385801,59662140581960376,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.366,1997-03-21,true,1318403061.0,10:56:11.11 +-1896561762,48369194630546440,NM,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.366,2009-08-14,true,1368212285.28,8:42:35.35 +-348506860,27061484627471976,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.366,1993-02-02,false,218579854.183,19:18:50.19 +321302176,15721618155621090,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.366,1957-09-26,false,764545053.054,14:52:27.35 +1640949311,23440185083097272,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.366,1993-05-03,true,550545864.886,18:37:6.17 +-1557426918,27784934057378376,NC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.366,1966-04-18,true,1121809247.32,16:43:20.54 +-953131,60690585663716832,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.367,2007-03-02,true,486826727.364,6:25:28.4 +-379775995,69395020404777680,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.367,1993-08-01,true,829262589.296,10:26:58.8 +778981983,79868502102063776,HI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.367,1951-02-27,false,1038044611.11,16:40:25.8 +822386079,66865947490357896,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.367,1972-09-19,false,286351714.555,22:32:20.17 +-397164330,44934798983698640,AL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.367,1950-10-01,false,555222097.455,21:6:42.30 +622168774,4843109787818865,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.367,2003-04-12,true,1324014194.89,1:30:50.23 +-625478436,475343221458688,KY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.367,1969-04-19,true,1364273605.64,14:37:58.48 +210668846,42233292775019480,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.368,1992-03-26,false,664872671.312,15:1:25.57 +429826297,51114854831304504,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.368,1968-06-14,false,1017336486.84,12:51:5.30 +731243063,1637817791053240,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.368,1975-01-07,false,1285046849.6,1:36:24.14 +-978382140,54567776811235240,KS,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.368,1963-01-13,false,1048651996.05,8:32:48.38 +-859019563,9626173920106364,OH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.368,1957-01-24,true,686453321.514,15:7:45.1 +1870292966,12914824769563014,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.368,2004-04-01,false,310488804.966,5:19:5.44 +416117351,29815477326874532,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.368,1955-05-21,true,52293417.9029,3:12:44.8 +-1088772985,53140941234078864,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.369,1980-05-27,true,897041321.684,12:19:3.25 +565081738,7596688176048272,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.369,1990-03-07,true,87914307.447,14:50:49.29 +1377300847,54914971557013072,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.369,1971-05-24,false,161121331.319,11:9:19.42 +1984439060,67740489413608664,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.369,1988-06-07,true,137075270.348,14:19:32.40 +329340531,11972425857717248,OH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.369,1950-02-16,true,897031045.149,20:13:24.13 +-1646741707,37790286468201128,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.369,2006-01-06,true,613132111.454,2:21:55.31 +-2128889459,5865282095951401,SC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.369,2006-02-26,true,444900178.482,9:32:27.15 +-2104979095,47597823064655392,AZ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.369,1969-10-16,false,511655861.019,20:22:53.10 +202229746,60388626471543168,NY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.370,1965-06-08,true,584922376.994,13:41:49.37 +801358554,43292552540158528,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.370,1952-01-03,true,1157126523.88,4:52:55.5 +-1507909706,58865809036477576,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.370,2007-06-01,true,35134524.6431,1:34:45.55 +-605176612,88165092531828352,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.370,1967-09-17,true,138928859.011,13:56:26.45 +858761102,25420278088832780,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.370,1950-11-10,false,1220603279.15,14:21:58.13 +1641961416,72550150906478464,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.370,1958-06-02,true,234878619.432,3:35:51.18 +754306383,67047844421174656,VA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.370,2003-08-06,true,613828802.012,3:37:21.14 +-1686723431,58552792065726232,AK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.370,1967-04-06,true,1261259387.6,14:47:57.27 +-636108222,24649384354545020,ID,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.371,1961-01-12,false,560567847.148,13:31:58.48 +991916622,17677550406442322,TX,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.371,1956-10-13,false,1096108281.61,14:43:27.37 +-292824557,41025022529521048,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.371,1956-07-14,true,177477875.887,4:53:46.39 +157574980,55652782979150440,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.371,1956-10-22,true,779716348.931,22:12:29.8 +-1280012558,63800536641064664,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.371,1957-01-08,true,176458087.216,1:27:13.38 +-1548534948,40177889080807104,ID,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.371,1961-06-02,true,1020733447.08,15:42:53.36 +-1890778914,25300257956079832,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.371,2008-05-18,true,345127344.938,12:22:35.37 +-392202664,19183267869995448,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.372,1977-08-20,true,53002768.9254,15:57:20.26 +-783284577,34783020146545880,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.372,1998-01-09,false,936957051.834,21:19:26.24 +-2096954649,65985205949749272,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.372,1980-10-23,true,200439874.958,9:40:19.46 +-532520480,35859515876041860,CO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.372,2002-08-22,false,935751670.568,8:24:56.9 +-175084841,77299217228519472,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.372,1989-09-16,true,1027099686.07,6:5:40.50 +-757522149,62875002986638504,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.372,1951-11-03,false,603456688.99,4:5:49.41 +2072935195,56111213232461128,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.372,1988-10-10,false,1094679179.09,12:20:16.57 +1763575380,6380072046099640,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.372,1978-11-03,true,160929461.422,19:32:36.37 +-1771286834,45087162821143904,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.373,1964-02-27,true,64492320.5727,13:26:48.51 +-1298470522,17308127366255800,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.373,1967-07-17,true,1335491919.24,1:54:8.4 +127278741,54785638043161664,GA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.373,1990-04-14,true,1261857458.67,19:1:17.28 +-1559724121,55566760456374448,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.373,1993-01-03,true,209225515.372,14:53:38.36 +-1204097754,53824944455548152,MS,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.373,1974-03-25,true,1147566935.6,15:44:36.38 +-1489640357,6327641699839416,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.373,1976-04-11,true,1309858.3706,2:48:6.11 +118458261,18394559126229832,NE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.373,1978-10-21,false,124792494.18,10:29:27.29 +-683307829,19397572328631952,DE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.374,1984-10-17,true,685628530.214,6:12:13.48 +1400346599,76499384066150608,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.374,1990-01-25,false,527275542.73,17:6:32.58 +-604911698,82404418762625392,ME,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.374,1974-09-10,false,962306324.521,20:21:5.1 +-1449317586,11893338185945130,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.374,1989-11-10,true,235443563.986,1:20:48.21 +-797670255,34316311204912272,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.374,1972-07-19,true,350323064.393,22:58:31.52 +-1616497228,59715439261344208,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.374,2009-09-10,false,1131521400.67,22:28:21.33 +-1786634119,46050237187112480,IN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.374,1997-01-08,false,218068198.662,10:5:50.14 +2032325094,36574419252010416,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.374,1994-08-03,true,799481281.097,7:36:49.21 +-1065608592,14109113774283868,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.375,2004-09-20,true,1045608854.6,2:58:36.6 +956481720,28509760198172028,LA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.375,1979-07-22,true,183867850.458,10:29:6.30 +973173187,11792299980998350,ND,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.375,2012-02-11,true,633048540.983,9:52:27.20 +-250904167,318337275684218,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.375,1965-06-26,false,576444511.56,14:26:28.46 +-506257880,23346792247990200,OK,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.375,1968-03-16,false,871110507.649,5:26:49.1 +-586482579,62952002621683464,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.375,1951-05-22,true,820867734.94,11:51:36.10 +-1029009084,30375747703589696,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.375,1971-10-04,true,850090054.658,18:22:40.3 +940764969,7290848495395011,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.375,1957-05-03,false,944217587.323,14:49:40.4 +1396187914,39605972286019592,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.376,1959-10-11,true,772097191.85,10:19:27.14 +1232909895,81348352146921184,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.376,1978-10-22,false,696011252.96,2:39:54.38 +950023084,19968175446374216,VT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.376,1994-04-07,false,99662450.6488,20:58:21.39 +-714001793,77935949363419968,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.376,1995-03-02,true,1298315401.66,18:30:20.13 +-1307308856,25663290068653536,NC,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.376,2008-07-15,true,941351478.916,22:2:45.6 +1275006019,75510860549117232,MO,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.376,1956-01-23,true,1007106689.32,20:6:42.53 +1236137202,24529542845068872,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.376,2006-11-12,true,726860687.226,20:15:24.6 +-52047129,4615219069767035,CA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.377,1978-07-27,false,525771792.334,1:8:31.24 +1350997629,9131175116897434,NV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.377,1954-10-18,true,1249806759.45,9:20:16.20 +1844063445,46643029403224280,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.377,1961-04-26,true,569321326.377,6:9:14.18 +-1778687497,45527993161262400,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.377,1951-10-02,true,310596237.993,7:22:20.55 +-936053068,55595123244893440,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.377,2011-03-07,false,816754346.643,6:1:22.8 +501421521,10243129147558012,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.377,1957-06-14,true,811741711.439,19:40:39.49 +-1430306290,55735442635427,OR,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.377,1952-04-19,true,240809441.729,12:7:52.52 +-67169477,13613612103768330,MN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.377,2003-09-26,false,872945484.768,20:15:36.55 +330487831,72632444543510496,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.378,1961-11-01,false,611589082.502,8:26:28.51 +292978793,72578239882445792,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.378,1991-11-13,false,120445986.231,20:27:12.50 +-1791291392,42585050069823264,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.378,1957-07-03,true,1187915075.44,7:13:48.5 +-1377140631,21963714085028412,NV,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.378,1968-11-11,true,568284448.109,8:5:3.18 +-934281151,83203889527717360,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.378,1966-11-18,true,1052005741.65,5:34:1.54 +-1227735142,82156633602190352,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.378,1966-03-25,true,934577636.576,6:36:12.29 +-1708993906,46191928538640528,NH,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.378,1966-04-09,false,436041572.154,21:8:35.44 +1917870538,74523140271188832,CT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.378,1953-05-07,false,1403842413.59,20:31:30.58 +-177368562,17028407227896176,MT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.379,1985-02-12,true,1208865342.22,4:26:3.6 +-1651834884,6832154767737621,MD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.379,2004-07-26,true,1237024339.04,20:20:20.46 +-4987512,16120674622413424,TX,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.379,1955-05-22,false,575103214.529,11:34:14.24 +1418707812,4707177063300721,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.379,1992-01-25,false,414148167.764,14:57:19.27 +-1354707072,20483367302531432,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.379,1964-11-26,true,593689248.843,16:38:39.55 +-126465299,32159186920375584,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.379,1983-01-20,true,1218585908.3,19:3:52.4 +1937741418,26362469948500264,SC,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.379,2010-02-10,false,1221987768.35,1:12:20.58 +-113829781,58930711180260488,CA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.380,1990-05-11,true,1023647826.95,19:25:26.16 +-1144548655,57990487199924208,MD,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.380,1973-07-10,false,1360117000.81,19:25:42.20 +-1535479719,46112920651322848,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.380,1978-10-15,false,372609584.478,1:23:22.2 +1983514365,84125669646200896,TN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.380,1986-06-05,true,1237772052.87,3:16:44.44 +1712652863,83612185491266592,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.380,1958-09-26,true,1312855529.5,15:32:47.39 +1109902771,7094100332045957,HI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.380,1951-11-09,false,1249364114.36,9:33:18.16 +-1159132559,81527029919331648,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.380,1985-01-24,true,737709316.47,5:26:23.34 +-2022834222,63090854787498904,GA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.381,1974-01-07,true,715018501.954,17:44:53.37 +-1006293516,34296839604684820,KY,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.381,2002-05-02,true,976163919.98,17:2:31.21 +1639370601,47775233200185912,MI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.381,2002-09-06,true,1065461337.25,2:55:27.53 +148958534,91902994729978096,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.381,1975-10-21,true,81869585.044,8:50:6.37 +1252620161,24731843622962984,FL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.381,1950-05-14,true,715390471.84,1:21:52.18 +1933976376,46251169055301120,IN,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.381,1964-01-24,false,511499156.856,16:50:2.7 +-340124493,10201314716428902,VT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.381,1970-03-05,false,1137712532.69,17:15:21.18 +2145312027,14653397099622830,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.382,2002-04-04,true,645945569.512,9:35:7.22 +-298262081,79551498692978288,PA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.382,2002-04-26,false,1106910412.77,17:25:19.53 +635665322,3856910466488832,VA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.382,1986-01-02,false,526439187.62,21:3:4.40 +-2064595753,65097589511248488,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.382,1973-11-09,true,461375370.646,1:49:19.19 +1065726011,46629886166591560,MN,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.382,1975-04-07,false,640023167.751,11:47:6.18 +-395411191,40158684922730264,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.382,1960-04-25,false,54169369.0832,2:22:13.54 +788103682,74773503982517952,MS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.382,1985-06-27,true,1330803246.27,18:7:8.16 +-1513157799,15038613582063032,DE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.382,1967-01-27,true,1016928199.2,18:18:32.42 +-248609967,59366385877174736,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.383,1977-07-04,false,1318785630.5,8:45:29.32 +-2024647370,5355219794072136,KY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.383,2002-08-02,false,1280272055.01,17:50:6.36 +741179009,65186158923084168,NY,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.383,1989-10-01,false,120716952.292,5:28:23.39 +-1430414881,80706079239079264,WY,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.383,1956-06-15,true,43516501.8404,18:11:25.36 +-1299785073,33233130527969668,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.383,1999-05-16,false,399703274.775,8:18:39.48 +1534075756,88650293226412544,MD,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.383,1956-05-16,false,1374566940.86,4:31:10.46 +-406911442,71296786043335576,MI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.383,1995-07-15,true,567449273.246,21:20:35.54 +1377905327,24717563023302512,KS,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.384,1965-11-18,true,297165918.142,11:29:14.47 +-340973482,8485198028878705,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.384,1951-07-26,false,309452402.639,5:46:36.42 +-2080875005,10873685140732784,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.384,1962-09-18,true,759497413.334,13:32:26.53 +98121783,82863151533621648,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.384,1962-06-08,true,799603694.138,1:10:28.43 +2042704813,79395011905889296,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.384,1969-07-18,false,1348870370.78,7:47:44.10 +753565880,22186002700380384,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.384,1980-11-05,false,566304204.577,16:16:6.31 +666345191,83464657411028640,WA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.384,2012-06-21,true,244041779.289,9:23:45.20 +-1892616672,35439048610237084,AZ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.385,1975-05-27,true,1134937478.89,17:5:38.30 +1010591196,17381030437805272,NH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.385,2012-07-11,true,73944993.2181,1:7:23.27 +-1690108596,58638831557807696,NE,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.385,2011-05-17,true,463376872.887,12:34:32.38 +764375438,54372155861516768,OR,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.385,1966-05-03,false,878722057.811,14:33:46.35 +995944449,27565688011636008,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.385,1982-09-22,true,946690563.289,10:34:41.50 +-53202426,28013272568943268,DE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.385,1990-01-22,false,233034079.429,16:33:19.39 +-378063895,86864705228061520,LA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.385,1994-11-23,false,284977509.502,16:36:27.39 +1720926943,6129961513343160,OH,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.385,2004-05-08,true,1298026805.35,10:42:31.51 +1281398996,59062757296078288,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.386,1958-05-14,false,921479722.305,7:51:23.5 +-1147979407,75265866540334736,NC,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.386,2003-04-01,false,605633504.318,9:40:20.8 +218439174,43078008032972384,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.386,1980-06-03,false,662853630.038,6:56:46.7 +636807463,70067253672206664,IA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.386,1970-01-13,true,611446336.354,8:1:46.18 +1091855529,778988678323630,MA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.386,1963-03-13,true,1299826140.71,15:28:12.8 +-917527086,57854083109914464,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.386,1955-08-12,true,1188164991.27,2:24:45.39 +-1128952705,38230678661459896,CT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.386,1977-11-14,true,68454602.6043,13:28:32.21 +3078550,70857910271866880,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.387,1985-07-10,true,1086696713.78,4:11:24.30 +1579399914,56710524028574560,NM,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.387,1951-04-26,false,1242611534.98,7:14:52.12 +498080035,8185027140203182,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.387,2003-08-14,false,424870828.394,2:17:7.36 +-623273841,19868611255384832,AZ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.387,1976-04-20,false,516941998.759,20:49:27.23 +165812879,31100758248836884,NE,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.387,1983-02-03,false,1312497323.83,9:17:41.53 +-314275714,39898647599507640,NE,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.387,1981-10-18,true,357831019.418,22:16:31.10 +-2119219541,12323907181423340,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.387,1960-06-04,false,925886457.689,16:16:49.3 +2024814458,39399700920588032,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.388,1976-05-19,true,1078289883.96,10:51:2.38 +-1699262224,69530554024167304,WV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.388,1967-08-23,true,780854929.377,11:53:27.46 +-1718043344,25397312088085280,WI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.388,1991-09-11,true,615434081.485,9:49:28.43 +932758684,4188171983574036,NJ,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.388,1981-11-22,false,261806272.536,6:8:32.58 +1713079839,90826320193080656,AK,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.388,2009-03-03,true,582549181.278,12:3:11.11 +-898918030,27675740116496764,FL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.388,1971-03-10,false,214328379.525,12:55:35.26 +1151255109,63794242329166144,MA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.388,1978-07-18,true,1168761591.19,9:23:44.58 +524320624,21099934126731592,ND,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.389,2007-08-10,false,154463003.805,9:5:11.38 +-1850817956,32605143311748956,MN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.389,1960-07-12,true,1177123337.4,3:45:49.16 +-829763346,66717461385899272,RI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.389,1984-01-06,false,539151662.78,20:50:47.46 +1479816017,59934919426028032,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.389,1964-05-24,true,217719512.54,14:26:1.47 +690490337,79043203977484656,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.389,1990-05-02,false,1029996149.58,14:54:51.13 +328123443,58942386658387696,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.389,2008-06-23,false,839798783.147,2:31:56.43 +-1214964473,31373273450640404,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.389,1951-02-07,true,286805994.208,17:49:20.16 +139520861,53802655045681872,LA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.389,1985-03-24,false,1151016570.53,21:33:46.2 +-552437760,42221719418459608,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.390,1988-07-13,false,3774537.265,16:37:20.22 +-358265760,41194782238803600,SC,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.390,1971-09-14,true,1268177958.5,15:29:25.7 +1205063808,15066647779418050,IL,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.390,1995-08-16,false,999027665.316,19:13:24.30 +-1997859571,82206755438927200,PA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.390,1980-11-01,false,1333013376.09,20:23:4.23 +-1033770877,36084820756044224,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.390,1979-05-25,false,822318673.669,15:20:35.29 +-261876905,77286651031979728,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.390,1966-01-27,true,518378096.947,15:1:47.50 +2144853235,64749953464433944,NJ,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.390,1973-09-19,true,491156458.28,1:28:37.6 +-1951263631,85626116851446192,AL,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.391,1980-02-01,false,749417286.25,21:30:8.12 +45545143,86679500628599712,RI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.391,2004-05-15,true,1071795185.98,18:23:3.3 +527467733,78187488794045232,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.391,1995-10-16,true,929181554.58,13:14:52.38 +896691951,49151218250379712,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.391,1997-06-02,true,98685828.8649,15:57:48.50 +-1850419228,56325728364793656,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.391,1984-04-26,false,554318123.722,6:7:18.41 +1953527280,77564657722070688,NH,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.391,1957-05-03,true,579792959.375,2:8:34.57 +1488934637,79263193990928112,VT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.392,2001-10-20,true,1166389066.0,22:3:54.38 +1060752521,603076328641689,ME,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.392,1970-01-17,true,1059607867.96,21:17:32.57 +-337862723,57502975139538136,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.392,1986-08-11,false,550622689.976,4:14:51.10 +903745700,2405963852834703,MT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.392,1962-10-07,false,21236197.4503,8:37:7.12 +225827351,62233322587162472,VT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.392,1978-03-26,false,1164121309.72,13:55:12.19 +-1428346236,70841428163219264,NM,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.392,1984-07-12,true,277506915.928,4:26:33.42 +-989782066,26049375059520852,DE,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.392,2010-02-25,true,1124540345.98,14:7:49.42 +-1863147111,605575932625039,MO,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.393,1980-09-25,true,841735082.089,11:47:5.55 +-1280047346,11355767064498196,OH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.393,1999-03-27,false,504275355.149,7:14:24.43 +-1056174558,36572129680572384,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.393,1964-03-04,true,98226482.7252,11:16:11.21 +-1034042877,30111727226076876,CO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2006-06-02 00:39:00.393,1953-09-06,false,1173293665.65,14:45:25.35 +-1049203650,80744428901708080,UT,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.393,1951-08-03,true,752876174.561,6:11:30.50 +-1607455859,47598345526696056,ME,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.393,1961-01-20,false,1052600760.26,16:58:36.50 +1353353489,63190805978262584,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.393,1976-10-01,true,1047529748.96,16:17:6.7 +62641256,25704655690761616,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.394,1975-08-02,false,206117594.829,11:25:39.15 +939152956,76641047375402880,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.394,1967-06-25,true,387501338.629,20:9:49.24 +2033862857,63920025823269072,PA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.394,2002-05-26,true,762640870.014,6:15:13.51 +996349382,89759143160447360,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.394,1985-10-10,false,95742484.032,6:12:50.36 +805493398,1383911177429155,WA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.394,1962-11-02,true,135868536.84,14:45:2.20 +-1796811376,61413785142287896,TN,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.394,1981-04-25,false,196024332.643,2:50:42.6 +-618330374,64274836081855728,TN,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.394,1966-02-26,false,1269151362.16,16:5:3.5 +-574897019,74310890023645232,SD,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.394,2000-08-01,true,9960655.45744,22:40:2.52 +1472762625,21828955135651452,RI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2005-06-02 00:39:00.395,1980-01-24,false,280978370.237,20:41:27.46 +-1534889400,52271758188488000,UT,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.395,1984-08-21,false,854331390.068,4:58:8.16 +-1299343173,54470523041637192,WY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.395,1953-02-12,false,970524865.957,15:24:34.37 +-990754215,35555528353296376,CA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.395,1999-07-01,false,921571261.889,19:35:27.14 +120280924,49007634811466528,FL,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.395,1959-06-20,true,420926039.29,11:20:52.28 +798594473,40796235134314760,HI,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.395,2007-09-01,false,829002010.436,18:42:18.13 +1732812945,27156587087587340,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.395,1974-02-26,false,52062024.182,3:5:56.42 +-1775809448,31516391083072992,OK,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.396,1972-08-24,false,41044607.0875,13:58:40.50 +687174744,79812397349374832,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.396,1964-09-12,false,40591844.9812,4:26:48.55 +-1015654946,71280608576087704,IA,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.396,1959-03-16,true,930115897.11,9:13:9.7 +-1405855553,44616957909480016,KS,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.396,2004-01-08,false,621153609.921,1:23:48.15 +1790281508,13847482286505544,CT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.396,1995-10-23,false,398271135.748,5:1:7.43 +-1394957273,32193703552823388,ME,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.396,1955-03-01,false,915919118.04,15:29:23.4 +-1384300062,24121065525215828,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.396,1957-02-09,true,135418768.657,7:8:30.3 +1455687901,46738404259040120,UT,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.397,1964-02-03,false,904652826.837,11:9:10.35 +196567829,80778905147986800,WI,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.397,1974-10-24,false,1094386599.78,9:46:13.28 +1714573481,54095790088215888,ND,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.397,2001-06-23,true,374267905.945,5:14:54.53 +182505467,54441564244478232,WV,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.397,2003-04-15,false,1193303173.19,10:57:24.7 +841251195,40398522082806712,NJ,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.397,1965-04-11,true,480019094.605,7:40:39.16 +449583199,21023781246653768,KY,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.397,1983-11-07,true,845239807.118,4:42:50.17 +-116208538,40811935774950208,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.397,1952-01-04,true,425354911.456,16:42:38.30 +-1163054544,77819583228797680,GA,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.398,1965-08-19,false,405785308.023,2:22:54.39 +190522724,34656683575222192,WA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.398,1996-07-14,true,26539778.1153,13:53:6.31 +-799146733,87368977763150208,WV,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.398,2007-06-02,false,557758217.471,14:38:54.11 +1579400696,57566000105278544,PA,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2010-06-02 00:39:00.398,1972-04-15,true,865779648.229,8:10:55.37 +532339912,83907873040863056,MD,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.398,2009-03-11,true,178745075.511,7:42:45.21 +-936167909,76683940233314848,NV,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.398,1978-10-26,false,67520809.6635,11:34:35.6 +1792133809,89836685802911232,MO,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2011-06-02 00:39:00.398,1994-01-05,false,154378837.741,5:37:7.38 +-28944165,20897206681556592,WI,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.398,1969-07-08,false,944412114.565,14:50:14.52 +-291204946,76707285048628592,ID,AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB,2015-06-02 00:39:00.399,1968-06-14,false,1374992377.26,5:25:46.12 +-1994453870,59637944849759336,NH,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.399,2008-07-24,false,885367435.906,9:54:14.58 +-49493094,26506558009694916,UT,GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI,2015-06-02 00:39:00.399,2007-08-04,true,930208195.245,21:32:16.9 +489219552,23245529806555248,MO,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2015-06-02 00:39:00.399,1995-03-04,false,365210706.887,15:10:4.35 +-679726102,66459754906061448,HI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2013-06-02 00:39:00.399,1968-09-08,true,565613009.341,10:7:50.56 +1963262483,80751794082361424,WI,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.399,1996-11-25,false,1192650888.1,4:36:19.57 +2052034343,66559539898021736,IL,DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF,2012-06-02 00:39:00.400,1969-05-21,false,68769002.544,16:4:54.7 +538251750,82943629001050000,LA,HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ,2015-06-02 00:39:00.400,1975-02-22,true,369067888.619,11:24:30.36 diff --git a/sql/src/test/resources/drill/window/datasources/allTypsUniq.parquet b/sql/src/test/resources/drill/window/datasources/allTypsUniq.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3819b215414bc1bdd00828535ff60aaa01b473f3 GIT binary patch literal 2364 zcmcgueQZ-z6hHU%zV5x!@=^B6s}w@wR_CZVSb?G=>ld)Sm2Pfj9|ej|T{MAgEEotz zx|ooRO<@M)VFP6Gl(;*pF< z8At+>h-5-y-l_^jXe&ffWz#fm-w75}O+=c4Gzp2dRY(~14ktau zVu{k`iK3l$j}ZkpDP~YOW)UoLS~w_*5EnDD)cBQc*WbeVNTWoQb>jQ44k zN-WnBWaDh~oEi_*-2{}OuYH%nkCp4auzQ(rzUj6vbH+)(+5gbL`bj(xdjx^Hx|k1O z(0}H%igrzf=!31P=gjF7{=R{@f8U8#zgDa+PPbb^dH2+fn=-&BvrT}b>fNKMVtKS& z+IL|q+3LlTfzu*PAT&`ZnuJ789Uf0g!^7E?-{@A9tOj0z9Z(m)!gUpSkQI< zNk)^m==b%Rn-3iGrUMZnGzQgMdIxfrsVlsw-FHR3>m{$|u;5!|S~=)ADOS^YHM}SE z=e7bo+ z&T|)QKIK+LdWZ?Y%Cc*VnO<4bs-py$!bC984Td2a#sMO9ulVuWV>CV9JN97?&Xjw%XK@~4|Jog$ za}hW1F5(Iqrk3!u65skk0h#WPC@{RlCAA7zTNU`@O9f6v75J%Lfy8}-6RU>o0Xfn$dlBw067-_(Vtg0AUxCWUoiY!u%EKo{}P|Ro1 ziXbsVIiE`xhKQLKhjDnZXhcP2gji`zvl}z=RmX5&12H@(QQ`rAY%0`w0D{C* zTbY#nI2t_O0K5l^j(dQ=3dI^MK>$Cw3N@Edh_J!0VF5xmE?zTP$TW)?D`BjZu~(Qy z8I&PF(3b}x)^CO|(^W7QWdkx)p7Mai*$)@d#`^j-QYLn?Ihnp*zcLRbni6M%B&5Vy zAemBP{*bLZd+d<)>}>0hW=^&lk5I~EE=cB-=*h;9M!GW(J2scyVc+W%9GJ~)FbZ}p z35m10Z0~dUa|#9j`KUQtIEO@a_A`_Ay$0Jk4s`|cXdfZxW$KUwo#U)@pk@wr7>Pap zx{reMhS4FJbPoCHNFNwbhmq0`>xdWJnGlCB$*qf&5HZq-N7MPwnBi-hosM+GfI5tn zejY}=aL%rA!z0bnMd})dkw&~7&La~X66n@6H|bDh3!$b!9=hnx7OO)t>kK_f!*iVN zb1bv9K-S-NXJ_Vc<9SKr{gUkxMO}gXe_upf`*fQNa!d`cHLiZ=%{4EqYJ`5gnPvFD F>0jLfPWJ!+ literal 0 HcmV?d00001 diff --git a/sql/src/test/resources/drill/window/datasources/allTypsUniq.parquet.json b/sql/src/test/resources/drill/window/datasources/allTypsUniq.parquet.json new file mode 100644 index 00000000000..abfffddc411 --- /dev/null +++ b/sql/src/test/resources/drill/window/datasources/allTypsUniq.parquet.json @@ -0,0 +1,22 @@ +{"col0":1,"col1":65534,"col2":256.0,"col3":1234.9,"col4":73578580,"col5":1393720082338,"col6":421185052800000,"col7":false,"col8":"CA","col9":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXZ"} +{"col0":2,"col1":10000000,"col2":-256.0,"col3":11.0,"col4":39598119,"col5":1388622482228,"col6":422086982400000,"col7":true,"col8":"WI","col9":"BXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXD"} +{"col0":3,"col1":-1,"col2":255.9993,"col3":0.0,"col4":82189300,"col5":1409617682616,"col6":422680464000000,"col7":false,"col8":"NY","col9":"CXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":4,"col1":0,"col2":128.9,"col3":1.0,"col4":77989310,"col5":1422836882404,"col6":421019078400000,"col7":true,"col8":"AZ","col9":"DXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":5,"col1":1,"col2":-128.0,"col3":-1.0,"col4":74969310,"col5":1404260882309,"col6":422434137600000,"col7":false,"col8":"TX","col9":"HXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXZ"} +{"col0":6,"col1":13,"col2":32.9,"col3":256.9,"col4":77761240,"col5":481249682638,"col6":422546803200000,"col7":true,"col8":"CO","col9":"IXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXD"} +{"col0":7,"col1":17,"col2":-33.0,"col3":33.9,"col4":47593130,"col5":1146529682748,"col6":422457724800000,"col7":false,"col8":"IA","col9":"UXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":8,"col1":23,"col2":13.79,"col3":-1.1,"col4":43932120,"col5":1117672082321,"col6":421673299200000,"col7":true,"col8":"SD","col9":"YXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":9,"col1":1000,"col2":29.13,"col3":0.9,"col4":54613101,"col5":-612747117889,"col6":422199820800000,"col7":false,"col8":"RI","col9":"TXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXD"} +{"col0":10,"col1":9999999,"col2":100.09,"col3":10000.0,"col4":85535120,"col5":-710119917582,"col6":420884294400000,"col7":true,"col8":"FL","col9":"EXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXZ"} +{"col0":2147483647,"col1":30,"col2":123.129,"col3":512.999,"col4":79322101,"col5":107828882418,"col6":422596656000000,"col7":false,"col8":"IN","col9":"WXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":65535,"col1":25,"col2":255.99,"col3":11111.11,"col4":84210222,"col5":707444882418,"col6":421353014400000,"col7":true,"col8":"MN","col9":"FXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":1073741823,"col1":1001,"col2":128.978,"col3":131313.19,"col4":59745100,"col5":770516882418,"col6":423167155200000,"col7":false,"col8":"MA","col9":"MXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":-1,"col1":-65535,"col2":13.9,"col3":10000.09,"col4":69885200,"col5":959905682418,"col6":423153676800000,"col7":true,"col8":"VT","col9":"KXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":-65535,"col1":5000,"col2":127.0,"col3":127.99,"col4":73220300,"col5":1022977682418,"col6":423132854400000,"col7":false,"col8":"NJ","col9":"OXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":10000000,"col1":3000,"col2":127.9,"col3":65534.99,"col4":55230230,"col5":1054513682418,"col6":423090777600000,"col7":true,"col8":"OR","col9":"QXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXZ"} +{"col0":13,"col1":200,"col2":1.0,"col3":-65534.0,"col4":69630500,"col5":1086136082418,"col6":421804713600000,"col7":false,"col8":"GA","col9":"PXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXD"} +{"col0":19,"col1":197,"col2":0.0,"col3":12345.987,"col4":62060100,"col5":1275438482118,"col6":422335987200000,"col7":true,"col8":"WY","col9":"LXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":29,"col1":4611686018427387903,"col2":-1.0,"col3":100.9,"col4":58576160,"col5":1306974482218,"col6":421473196800000,"col7":false,"col8":"NC","col9":"NXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":23,"col1":9223372036854775806,"col2":99.9,"col3":303.12,"col4":70230300,"col5":1338596882418,"col6":422924803200000,"col7":true,"col8":"SC","col9":"SXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB"} +{"col0":0,"col1":9223372036854775807,"col2":63.99,"col3":9898.68,"col4":80611100,"col5":1370132882318,"col6":422613849600000,"col7":false,"col8":"KS","col9":"ZXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXZ"} +{"col0":109,"col1":92233720385475807,"col2":69.89,"col3":9798.68,"col4":81210180,"col5":1438547282318,"col6":422648928000000,"col7":true,"col8":"NE","col9":"VXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXZ"} \ No newline at end of file diff --git a/sql/src/test/resources/drill/window/datasources/fewRowsAllData.parquet b/sql/src/test/resources/drill/window/datasources/fewRowsAllData.parquet new file mode 100644 index 0000000000000000000000000000000000000000..640afe51f34e8e3e8628253fa8833e10da745f29 GIT binary patch literal 3990 zcmcgvdsr3K7GHa256sXxdJcJLR3btkj#^Ip3mMp^d~j4z&LlB_Vc;0% zj_?6^p-53wy11=fDK!B2^aQZ)4^WA^1_9u^Ab=NR0Nif0D>M2K&|`da1b|IHfSY>} zAL1CT1qfm1LoE1r#`SMg-dnXfu&=5)Mn>Qt0iGNW*;jG4*tbRf^YL_yIKK6*3Z zMBXpaPPM4bL|y$B@*%E8&SqOdN5kS!lAFfa;%W)+X`twmmebR~Bb+S{Ar1mh*SQFZ zyv2)0soiox$2+G8m@%W4L%(B&&qbbu-)uheW^R)7ykVu`cp}ppa(8y~7SWeGZjZDYTwf%)$TGAjC(a#uC83A&}3>dWQ3$n~FGWMN*N56R4*@`uas;K$9_nhgA!WCc11a_Yx0{c7jE4$1+RW zY`ESue}js&DMIK&MOUmU_YlI}7AYiC(-y=>Ek-?Uy?}|DR-mGC4w~>+get_`auG2% zUKJ|+EziWMO9uP0I&OdqTWo9i58HiVnS@ayBSmy9=0(=ix|r%xkBlcaG&+RXz;Mon zMnw}1jgoK{&S@}Xa161h3E?=8;`$5zf(@8B5yChhAxO~Bgi)L)=Oo0_m^jX!>kn~) z$c;w1G@3XGu4tZtV+tJ}Ok4#U!Ief2MY9#c4?;9f^XZ6q;tJ7RK1Pk?KBiIx5owGG zbH#G@LJ)*-@q$0%5NX74;%a1sI$^u9hp=7{g%uFP#Y6P-Dy(=R`jycj6bd57szbG_ z;6xKfl5ArSW0A1QJ6KZ7j2fzr(I#xp_7KBy+Pyj6ro^XR;ZIY@qPO%kU1IRlG(jB; zu+pV#0lX1YNTM14wL{Il0YZr%gqfXyCYsEB0K$piSalL^#-m3jes!AwB=bm=1gq*G zVrj33@_MKfNxL6({<4wDCFQxK&XcLbv6ew?odw#_rsJ25(^yjH2RL@RtVLeOszNmD zh@(nM*hhQ|u_oFCb;%5x)p^i%3zVaMB`L2|9aZ<1jTfUFNZizg1sK;FoHert!fQ5|m?4CB{<|W|6Qy^462t%=;I39Q@LE5EPVcNu!dWcb)9O&- zf%m`w5uiouY(~5LT4#8m=e5at9kgo24r5h(NS6WJpI|g*I;-`l8S7PdjNA zqKTO(Lj`75Y2qpwywPUTO3a7ImM)bc@_iY;!Po}GpcYn+=r3Y!|79|G6w6>U33FOx zShzxl7trR)whXtCOh00Im_BlW45u;G+p}e`FOp&3 zEEz^3l&JSGB^X=`?%fX!LPXNpT796kv#iUH&)#W$5O7%R+`J7!cL&~d<<}MGf_|1n z{&Gy-%J=PWRaiG~li3;|^N-N<-#^_g_1d0L{EyJH()yw4dx~RENukBVzq?wqS9;-- z0dw;r%A|vLT7u?$a$FiRxP1TeSIeZh67#bf=_hG>*p;HZ^WtLKImNjDx$3K{z6Zp`iU?M}@`#5yy0QuY~)JGxI;T#Xbx zHYaUI!!gP3!npB`g%wiYwdPWv;!Lce-^B^;ClKSi2aereBP}^#@J;G@Rx*xx=c;SL zX{jjHbbD;CJrY1tTFBBGY0Q$Ve{cQ$gfzRCQ`+9erx8z8-?!IQO9!=leB$fgx#o;j zs0%0RzV&>4MU_+RC`49Zg z!_t5S8-J_bRVB4Nu~Tu_W_6)2KCA>NdyG=YZLRNQzsoG zzrTJ?`rxhpJ9&?vv9w_4Alh~)%POA;IAZE^O}p>f@9L+l%p4W-aITF zc8>B#!9K^kLh&HQQ8vka@nE|Vgs;zT-@AfHNGOv9tdDV!AjqCZ(qf)`mtcubdyQE0p zV?{TZEx7!Kcf-CoC;c@nxZywe*@>CAGdZJElb6yKX8@At@tjL#g1N@@jx zPWcR_3X@fP3g#z-u?~NZr*Y}0@bUC5sa>?5rUrZnYK>nPG&5P3o@4HAHWTJQ1#Dt? z00TXqV?TP^pQ@BgReG?y*Of1c$Jx3WH-t09ZsORj#r3W7Flg(SCutWVn*Vi`V$ zjhx@grn*j`ImSwwKk@)MFntiSs&OM2nBLn=G&_3P@Ip_HQ1 zU+n$@>V8Q7xV2ZOb#bV}85>mki~sL8GWQkYd9fBQHfoC;z{hj)5XyM-FoHI=`q^Dj4FTe(SyZ3zn`PQuA8OrLsVsUs6IX9>q`f>f753vRb1Z(5{}=W);~@Y5 literal 0 HcmV?d00001 diff --git a/sql/src/test/resources/drill/window/datasources/fewRowsAllData.parquet.json b/sql/src/test/resources/drill/window/datasources/fewRowsAllData.parquet.json new file mode 100644 index 00000000000..2dffec37526 --- /dev/null +++ b/sql/src/test/resources/drill/window/datasources/fewRowsAllData.parquet.json @@ -0,0 +1,78 @@ +{"col0":12024,"col1":307168,"col2":"VT","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1338596882419,"col5":422705433600000,"col6":true,"col7":3.95110006277E8,"col8":67465430} +{"col0":-1704,"col1":48600128,"col2":"ND","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882420,"col5":422407699200000,"col6":true,"col7":1.29358204137E9,"col8":75128560} +{"col0":-19521,"col1":3640288,"col2":"SD","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1370132882420,"col5":421580246400000,"col6":false,"col7":9.83657842924E8,"col8":71170420} +{"col0":-4709,"col1":3783896,"col2":"MN","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1306974482420,"col5":422796153600000,"col6":false,"col7":4.66674349327E8,"col8":55294390} +{"col0":1600,"col1":8107680,"col2":"MA","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668882421,"col5":422410982400000,"col6":false,"col7":4.7471347117E8,"col8":37915240} +{"col0":-1950,"col1":40547891,"col2":"IN","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882421,"col5":421379884800000,"col6":false,"col7":8.89911941945E8,"col8":5867310} +{"col0":-1968,"col1":6212808,"col2":"SD","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882421,"col5":422410032000000,"col6":false,"col7":4.72880577743E8,"col8":22332400} +{"col0":-788,"col1":7171344,"col2":"CO","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882422,"col5":421916601600000,"col6":false,"col7":1.04871325474E9,"col8":34731100} +{"col0":-1846,"col1":5077360,"col2":"WI","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882422,"col5":422439753600000,"col6":false,"col7":1.4793288844E8,"col8":72125200} +{"col0":1001,"col1":7819200,"col2":"MD","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1370132882422,"col5":421684012800000,"col6":true,"col7":9.36437541786E8,"col8":75052800} +{"col0":19860,"col1":133164,"col2":"VT","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882422,"col5":421253049600000,"col6":false,"col7":1.09109659183E9,"col8":73602340} +{"col0":-1,"col1":-1,"col2":"VT","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882422,"col5":421253049600000,"col6":false,"col7":1.09109659183E9,"col8":73602340} +{"col0":1140,"col1":108878,"col2":"HI","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1338596882422,"col5":421429392000000,"col6":false,"col7":3.71555243082E8,"col8":7378300} +{"col0":-8984,"col1":677944,"col2":"GA","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1306974482422,"col5":422895340800000,"col6":true,"col7":1.42680533942E9,"col8":66670320} +{"col0":-14211,"col1":7985264,"col2":"MN","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1338596882423,"col5":422065296000000,"col6":false,"col7":3.90513711473E8,"col8":16118230} +{"col0":-19479,"col1":240352,"col2":"GA","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1338596882423,"col5":421930252800000,"col6":false,"col7":9.72747639559E8,"col8":63177700} +{"col0":4242,"col1":502144,"col2":"WY","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882423,"col5":421398806400000,"col6":true,"col7":1.13430261855E9,"col8":9238280} +{"col0":-3576,"col1":5392352,"col2":"MI","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1370132882423,"col5":421822339200000,"col6":false,"col7":4.70197757462E7,"col8":43115460} +{"col0":-1816,"col1":29832,"col2":"MA","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1338596882423,"col5":423068745600000,"col6":true,"col7":2.68051680205E8,"col8":8566390} +{"col0":362,"col1":24064,"col2":"OR","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668882423,"col5":423099072000000,"col6":false,"col7":3.84895861757E8,"col8":69763320} +{"col0":-1645,"col1":908720,"col2":"MA","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882424,"col5":421244323200000,"col6":true,"col7":2.73055641297E8,"col8":71068900} +{"col0":-1159,"col1":50752,"col2":"RI","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882424,"col5":421906147200000,"col6":true,"col7":4.01229124414E8,"col8":5649490} +{"col0":-202,"col1":5700,"col2":"MO","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882424,"col5":422671564800000,"col6":false,"col7":1.84965459559E8,"col8":29932570} +{"col0":-3399,"col1":74400,"col2":"RI","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882424,"col5":421853702400000,"col6":true,"col7":1.24109748587E9,"col8":40411560} +{"col0":564,"col1":537456,"col2":"AZ","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882425,"col5":421991683200000,"col6":true,"col7":1.09531044142E9,"col8":39161450} +{"col0":739,"col1":103690,"col2":"OR","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668882427,"col5":422474054400000,"col6":true,"col7":8.68835049003E8,"col8":17273570} +{"col0":14587,"col1":617504,"col2":"WY","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1306974482427,"col5":422182627200000,"col6":false,"col7":5.3037680439E8,"col8":81757180} +{"col0":-61299,"col1":383096,"col2":"CO","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668882427,"col5":421627161600000,"col6":false,"col7":2.69607476481E7,"col8":27971300} +{"col0":11345,"col1":8788,"col2":"MO","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668882428,"col5":422137872000000,"col6":true,"col7":9.11002550279E8,"col8":20510140} +{"col0":95690,"col1":189392,"col2":"VT","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1306974482428,"col5":422953401600000,"col6":false,"col7":1.25727344595E8,"col8":29546560} +{"col0":-199048,"col1":357520,"col2":"SC","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1370132882429,"col5":421109020800000,"col6":false,"col7":1.27909910034E9,"col8":51408700} +{"col0":216553,"col1":86744,"col2":"SC","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1338596882429,"col5":421599945600000,"col6":false,"col7":1.17889933749E9,"col8":53136170} +{"col0":-126232,"col1":38160,"col2":"WY","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882429,"col5":422958240000000,"col6":true,"col7":4.20766450283E8,"col8":62477170} +{"col0":-14479922,"col1":18402,"col2":"NE","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882429,"col5":423112896000000,"col6":false,"col7":1.1120849455E9,"col8":25806550} +{"col0":-185276,"col1":1150,"col2":"SD","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882430,"col5":421212009600000,"col6":true,"col7":9.3620425577E8,"col8":37363300} +{"col0":-14479,"col1":51192,"col2":"IN","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1338596882430,"col5":422682019200000,"col6":false,"col7":1.17748065647E9,"col8":12482490} +{"col0":155858,"col1":46640,"col2":"RI","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882430,"col5":422123356800000,"col6":true,"col7":1.02765112757E9,"col8":40761560} +{"col0":1,"col1":0,"col2":"RI","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882430,"col5":422123356800000,"col6":true,"col7":1.02765112757E9,"col8":40761560} +{"col0":-823,"col1":6352,"col2":"MN","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882430,"col5":421587244800000,"col6":true,"col7":1.02635426051E8,"col8":17111380} +{"col0":13899,"col1":29548,"col2":"UT","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882431,"col5":421273872000000,"col6":true,"col7":1.12572986097E9,"col8":8261230} +{"col0":-20449,"col1":5872,"col2":"NY","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1370132882431,"col5":421681680000000,"col6":false,"col7":4.22305438068E8,"col8":18255480} +{"col0":-20449,"col1":5872,"col2":"NY","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1370132882431,"col5":421681680000000,"col6":false,"col7":1.0,"col8":18255480} +{"col0":-86821,"col1":115648,"col2":"LA","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882431,"col5":422381779200000,"col6":true,"col7":1.26712293161E9,"col8":82189300} +{"col0":5562,"col1":58144,"col2":"AK","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882431,"col5":421205184000000,"col6":true,"col7":8.30455866482E8,"col8":42999360} +{"col0":3894,"col1":247308,"col2":"OH","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882431,"col5":422359920000000,"col6":true,"col7":7.63454104971E8,"col8":50267140} +{"col0":10770,"col1":212492,"col2":"PA","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1370132882432,"col5":422397417600000,"col6":true,"col7":7.2797834549E8,"col8":13490300} +{"col0":14016,"col1":7755464,"col2":"IN","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882432,"col5":422252870400000,"col6":true,"col7":8.28874169301E8,"col8":55007110} +{"col0":2532,"col1":338648,"col2":"DE","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668882432,"col5":421450560000000,"col6":true,"col7":3.99985456569E8,"col8":73722390} +{"col0":17069,"col1":648376,"col2":"MN","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1370132882432,"col5":422475696000000,"col6":true,"col7":1.42275732068E9,"col8":13465170} +{"col0":20257,"col1":195470,"col2":"IA","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1306974482432,"col5":421379452800000,"col6":true,"col7":1.23097632842E8,"col8":27742430} +{"col0":-71899,"col1":104126,"col2":"KS","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882432,"col5":421108329600000,"col6":false,"col7":0.303,"col8":77761240} +{"col0":72309,"col1":752768,"col2":"WI","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1306974482433,"col5":422809632000000,"col6":true,"col7":6.16602756464E7,"col8":8470130} +{"col0":11281,"col1":45224,"col2":"GA","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1338596882433,"col5":421905283200000,"col6":true,"col7":1.40238980845E9,"col8":5527400} +{"col0":19294,"col1":831584,"col2":"AZ","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882433,"col5":422289676800000,"col6":true,"col7":1.41978080866E9,"col8":49582500} +{"col0":8966,"col1":80944,"col2":"MD","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882434,"col5":423039974400000,"col6":true,"col7":5.6890579253E8,"col8":35118460} +{"col0":-17427,"col1":85048,"col2":"MO","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882434,"col5":421333920000000,"col6":true,"col7":4.6673697742E8,"col8":22981110} +{"col0":-110,"col1":2632,"col2":"GA","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882434,"col5":421494451200000,"col6":false,"col7":1.44804901984E8,"col8":59898370} +{"col0":3418,"col1":493976,"col2":"WY","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668882434,"col5":422340566400000,"col6":true,"col7":8.56574532391E8,"col8":37636440} +{"col0":-446,"col1":25236,"col2":"OH","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882434,"col5":421380316800000,"col6":true,"col7":7.35715255558E8,"col8":51842560} +{"col0":5530,"col1":199400,"col2":"NH","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1370132882435,"col5":421374960000000,"col6":true,"col7":8.91015086626E8,"col8":71064450} +{"col0":0,"col1":1,"col2":"NH","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1370132882435,"col5":421374960000000,"col6":true,"col7":8.91015086626E8,"col8":71064450} +{"col0":-977,"col1":458168,"col2":"ME","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668882435,"col5":421534108800000,"col6":true,"col7":6.677148957E7,"col8":63195250} +{"col0":-1727,"col1":41872,"col2":"IA","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1338596882435,"col5":421150492800000,"col6":false,"col7":1.37327670868E9,"col8":58690170} +{"col0":-8117,"col1":84976,"col2":"MN","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1306974482435,"col5":421977081600000,"col6":false,"col7":5.0893542491E8,"col8":52269240} +{"col0":-14062,"col1":74776,"col2":"IA","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882436,"col5":422806694400000,"col6":true,"col7":1.38237309946E9,"col8":66897160} +{"col0":1919,"col1":50312,"col2":"UT","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882436,"col5":421410297600000,"col6":false,"col7":8.02591009467E7,"col8":10039350} +{"col0":15935,"col1":806060,"col2":"IA","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1401668882436,"col5":421357766400000,"col6":true,"col7":-1.067,"col8":29033340} +{"col0":374,"col1":43592,"col2":"NC","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882436,"col5":421799529600000,"col6":false,"col7":3.06514832189E7,"col8":9995490} +{"col0":1868,"col1":290820,"col2":"NY","col3":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col4":1401668882436,"col5":421818796800000,"col6":false,"col7":1.02289267124E9,"col8":11248450} +{"col0":11107,"col1":186364,"col2":"MN","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668882436,"col5":423020620800000,"col6":false,"col7":7.91697316619E8,"col8":53224800} +{"col0":63389,"col1":38640,"col2":"UT","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1433204882437,"col5":421578691200000,"col6":true,"col7":1.00485076217E9,"col8":22871350} +{"col0":15426,"col1":32192,"col2":"WV","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1401668882437,"col5":422003520000000,"col6":true,"col7":1.0252414664E9,"col8":56382320} +{"col0":-292190,"col1":385288,"col2":"CO","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1370132882437,"col5":422413142400000,"col6":false,"col7":8.0256170003E8,"col8":66881100} +{"col0":0,"col1":0,"col2":"ND","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1306974482437,"col5":421739395200000,"col6":false,"col7":3.30283711488E8,"col8":73737330} +{"col0":-1732385,"col1":56312,"col2":"IN","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1370132882437,"col5":422905363200000,"col6":false,"col7":2.18199000393E8,"col8":79702400} +{"col0":-955329,"col1":778984,"col2":"CA","col3":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col4":1275438482437,"col5":421725744000000,"col6":true,"col7":1.38262054833E9,"col8":29513360} +{"col0":1722833,"col1":5376880,"col2":"ME","col3":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col4":1401668892193,"col5":421672608000000,"col6":true,"col7":1.19950218188E9,"col8":36795330} +{"col0":-5186159,"col1":31933,"col2":"NH","col3":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col4":1275438492194,"col5":421295817600000,"col6":false,"col7":2.30933430625E8,"col8":10938520} \ No newline at end of file diff --git a/sql/src/test/resources/drill/window/datasources/forViewCrn.parquet b/sql/src/test/resources/drill/window/datasources/forViewCrn.parquet new file mode 100644 index 0000000000000000000000000000000000000000..4c1e70c45a35009090ec242b82c97e4421fa9da0 GIT binary patch literal 2383 zcmcgueNa?Y6hHUvd&}E(7k7!bK4>*n&p!42aWToNX?(F?I_nhB7 zzkANT_wCY{v^gAbLxgK2VVoU@YK}GYvQ1NUW?DfMLmW!t10ZNE)d0JYh)&Fq%{*BTRy;u}VmEc}K;y`g&*M zab7AsE3GdTnMFS_D>VGjHK;UxG?HPB1|tiso;R{a9@KI#7zr?X4aB3?21IW#+Kd^- z2xB7J+Ken|Aq1_7@iL6)iBT~7EAAvNy%cxHmkX7i$x|^|vj1ihsQ4#pIw=L470a#e*G$w$V1!)?d$cf&>b2M5DSd z@7Jq`QQdcGFf*PQbnEKlg9)Cqu!38$i%{Qxvvpn~VRaAg<;P-oHljKXySrzTBM2Mz z*<|OBi0$tjYWrk8R2oaMm+>dJbL~ zPtVmSNV(24SDlDKWV3T?vDLS9jR=z|!A0iFkDf^F?Lz zRbMH4-&-pJi*?TjBFuD&@Z>QOns6MvP6S<<2sM==97ox)O=M_y0`s8=H_B05Ekb&+ z2nQNOn6gWRp=~1cA>{FwMcDX-2$DC+*DqT|sQy@l9o0&0+aW^taZ$QbfgJCXBtm$i zFw=%8%+xcEPF1pb`H^g9JbS7myk|h**~88S+Ro?#!JAEYwgS|49&#?2dqFrD{Pa&9 zlX`@_oqc&gASE4tG_(u5i++1BbH!0%CTc$? z-wHE6EiJ5hX@d}bGxhcJOq(!IDit+CLEM@W_vuQeysprFSO`_WvUl6;6GC&6j#@T+ zLzr^?;=mPWgYd&()mpp%e!;stj%9MK`R#%M=||LF7nIy?^}YAB+fhODbXw#JZ*JX{ zN+eE34w7q5%p2-mgA3@!FBhu{cP*sCcV@yf7xFRr2e(!2Dt6#CXz!_^${M$Vr`k&2F0#uWgU@u3x{nGesjhwF1**B? z7~!a(`b4_4q{PJq07r#VykA~PQO=t6F5qaCw34)HO@Tb28=c52Tv?hkHx%D~X-X&A zl*w~Z^5jFHt9XN}*cs|wSuNS+yFzt%rVyr)mo6zOS`VBSW!Z85sv?DkZ0sry^#_wZ z$Q-oK!kUq56btI0hZH%=WH+@^rXgZs?f?1YSze{g9PNep>_b7(ruQkdu=5jrt1SCy zGpF-~E4!BACsn3TD73KizMmJ%IrSLik*Fvjv*`qlHg^91e*P`J+Hs*!GBX;AT{K!? zC?eplFZ(cFw%nci+9g z`+oN~Z{Ex%+LMNeAYRgmS9M{c3%h!e`Af8Em>wir@aTzZ0X!N>(>)G@(JJiN401J<`t8ZUX)#690x?#Dh5}fh<8vqG7Ha-?$Nu&;P0tg&v zB(T}6suRKhHoxDYUR*>#qxz3{9o+ZnPmH|YV!fh)Dx&;JHVeLSb%XzVt>!va{X|lxGr?{1^<$8IPC3+g` z#LL7Kyg0`YJtXsbi0365Zzn!NARQY=jDrG!j!r;RPbb9^Bf4#Z6CAt`jN_vPr(mR$ zCi9_uA0eG4B=J6cC?p9IKSeMiz&?c!IHoWCp<#v8sQv$Gle(>eS~weMmEmwxoQh&h*;hIX^L>9zU+v7nM08x z>@V}Bj((XZ+OsgFFpqo@$jhFZhHHFunlC^!iG(p}eSoIMr1=4iMRT0nbZBNQiJYPN zV$&i46cSlq2K5r@Xn=|acPA1$x}77j-%2W4N&OzK9b#W;wl^`yfE6^JU+&SISL3hj(cBuK#H#ZG-4SC2q<*p{-+r?jNI$n4>R zTi3^9sS<0Bf@39EdgoJ`=@H}KH>-v^(Epzsch(KT--mau^^GCS=!UVlX=SZZINF76 z{!82BvBW$D7w{vKw2DEmKgVCZf-5%|T1=5>`)|#%$p~mQg|EP%Hd8P>9?5|1D0oO} z;1`pG20SrIxSf9a>uuvOvfU&>`@l$Jd#U|$KUtJa0nkx_iQigR+lYxH6V>F}6*rN$ z`t#{{ZQD{8@AzamVUcbsXzp`xqi`5k81C0q-HWGrCluVTZ=@-nq8mx2r|9ja3NUO% zA1Dx#r$GM$3LMA2-|rPL<||ORT!Bx~PFt<;^gh2+f$s~@nx(*`Yz2xBD)8z?1)i){ zpcxUp-%?=FF$Jz3SD-CJfqSbISoyI6YgZz=Df;YM1sXn8V9pFogP}#c6qvPD0S8Vr zZL*y-Q+S%ld^gyCdZ`0ZUv5QQeX+vPM9swu+zwF z_MbcrHSiPrblW}%k%*^uQ@E#eeZT!b_>QjEo=1aNq3;&UdVMzT*4j;Amsp8wgxB?` zIdbPxX`1T6dR^b=8u8i%d5_=VA8LMeM?TveJoVt3^K$BMkJJph(;}DqPsxZ9m&r-T z`@Az?-(h*`)~e>#F=ymkOEC5FB6$eI2I-+3_Mc$%SFWVT1a7;E1l z&pVa(-qb5CGRdj=@nE(5Zr0y_37NQG4ngl1%I4v(gDQ1=*l_R`gw|(_+zWnt^ zF-L!M^Nj4ks(Ki!WUMhb&c8+`iAJ&XTbZ!L=6l;@#mEg<7r0Eut7X*S7A>ixu~05M zH-7ko*)4LH)ZQ#WEB*E>Rq|5eO}&eP%xq)NZ}^7bnzpm@45Svp{wM^{S9Xhx%!#v1 zUn(P=eXf!3J^l5p`{%z?9`1Mj*tPDvzEciIw2@jDuFA9}G_P$0J$hoiE^}iv zH)!2tc>O{K7{}yuO=nZc)Q8@j*39q5Q5gOQ8Ys@B%eK3q$=Rl>{Z}*G)m7_Q%O1kIW;$88(Y z_R{Q`nbUKpy%vO9h&E)@hVtEG!`y=!&K;MRy)ZA^71Wtnuf=8L1sQSYd73#VCu=^4 zUTC}L&6ttpVc`zW%MR)R7MHKT?=CCzN23yFv6gi8_Qba8uQY*9l&$p&^FZCB` zC*)ZfZqRLm^9?pCxL{v&TWs>2AI)ctG2T4H(zc}^4$(V>=$ zJjU3h|Mz>``jgKyCA!_1E-4;kEj}p4OWu6e`<7?gciu!#MbxsMq8GJ);k{vP^79wr zIqkI@?M5Y9QqXOap6#^O>fZjMSNEHX%b<(gHUqkmT-OHh!d`2w>;D3|yzDM=xGw7Y jBe@qIZcEwkEG`K4a!t=&uxM7^!0cSm>i`n*FU)@dL%Pwj literal 0 HcmV?d00001 diff --git a/sql/src/test/resources/drill/window/datasources/smlTbl.parquet.json b/sql/src/test/resources/drill/window/datasources/smlTbl.parquet.json new file mode 100644 index 00000000000..3b502952e30 --- /dev/null +++ b/sql/src/test/resources/drill/window/datasources/smlTbl.parquet.json @@ -0,0 +1,56 @@ +{"col_int":8122,"col_bgint":817200,"col_char_2":"IN","col_vchar_52":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col_tmstmp":1409617682418,"col_dt":422717616000000,"col_booln":false,"col_dbl":12900.48,"col_tm":33109170} +{"col_int":407024,"col_bgint":37168,"col_char_2":"VT","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1312244882419,"col_dt":422705433600000,"col_booln":true,"col_dbl":39006.277,"col_tm":67465430} +{"col_int":41704,"col_bgint":470128,"col_char_2":"ND","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1396398482420,"col_dt":422407699200000,"col_booln":true,"col_dbl":12041.37,"col_tm":75128560} +{"col_int":-19521,"col_bgint":30288,"col_char_2":"SD","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1425256082420,"col_dt":421580246400000,"col_booln":false,"col_dbl":97842.924,"col_tm":71170420} +{"col_int":-4709,"col_bgint":373896,"col_char_2":"MN","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1357086482420,"col_dt":422796153600000,"col_booln":false,"col_dbl":449.327,"col_tm":55294390} +{"col_int":16300,"col_bgint":81680,"col_char_2":"MA","col_vchar_52":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col_tmstmp":707444882421,"col_dt":422410982400000,"col_booln":false,"col_dbl":4771.17,"col_tm":37915240} +{"col_int":-15950,"col_bgint":4547891,"col_char_2":"IN","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882421,"col_dt":421379884800000,"col_booln":false,"col_dbl":8841.945,"col_tm":5867310} +{"col_int":8888,"col_bgint":314696,"col_char_2":"FL","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882421,"col_dt":421621977600000,"col_booln":false,"col_dbl":1183.32,"col_tm":6605110} +{"col_int":82588,"col_bgint":6672,"col_char_2":"ND","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882421,"col_dt":421788297600000,"col_booln":true,"col_dbl":436.897,"col_tm":41305100} +{"col_int":-13968,"col_bgint":62808,"col_char_2":"SD","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882421,"col_dt":422410032000000,"col_booln":false,"col_dbl":42577.743,"col_tm":22332400} +{"col_int":-74788,"col_bgint":7344,"col_char_2":"CO","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882422,"col_dt":421916601600000,"col_booln":false,"col_dbl":10254.74,"col_tm":34731100} +{"col_int":-1,"col_bgint":0,"col_char_2":"CO","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882422,"col_dt":421916601600000,"col_booln":false,"col_dbl":10254.74,"col_tm":34731100} +{"col_int":-10846,"col_bgint":57360,"col_char_2":"WI","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882422,"col_dt":422439753600000,"col_booln":false,"col_dbl":1488.44,"col_tm":72125200} +{"col_int":101701,"col_bgint":719200,"col_char_2":"MD","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882422,"col_dt":421684012800000,"col_booln":true,"col_dbl":97541.786,"col_tm":75052800} +{"col_int":198860,"col_bgint":103164,"col_char_2":"VT","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882422,"col_dt":421253049600000,"col_booln":false,"col_dbl":10591.83,"col_tm":73602340} +{"col_int":11140,"col_bgint":10878,"col_char_2":"HI","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":896747282422,"col_dt":421429392000000,"col_booln":false,"col_dbl":3243.082,"col_tm":7378300} +{"col_int":-84984,"col_bgint":67944,"col_char_2":"GA","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882422,"col_dt":422895340800000,"col_booln":true,"col_dbl":14339.42,"col_tm":66670320} +{"col_int":-184211,"col_bgint":85264,"col_char_2":"MN","col_vchar_52":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col_tmstmp":1401668882423,"col_dt":422065296000000,"col_booln":false,"col_dbl":311.473,"col_tm":16118230} +{"col_int":-169479,"col_bgint":540352,"col_char_2":"GA","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882423,"col_dt":421930252800000,"col_booln":false,"col_dbl":9739.559,"col_tm":63177700} +{"col_int":42942,"col_bgint":502144,"col_char_2":"WY","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1401668882423,"col_dt":421398806400000,"col_booln":true,"col_dbl":1118.55,"col_tm":9238280} +{"col_int":-33576,"col_bgint":592352,"col_char_2":"MI","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882423,"col_dt":421822339200000,"col_booln":false,"col_dbl":475.7462,"col_tm":43115460} +{"col_int":-127816,"col_bgint":9832,"col_char_2":"MA","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1401668882423,"col_dt":423068745600000,"col_booln":true,"col_dbl":2680.205,"col_tm":8566390} +{"col_int":36582,"col_bgint":243364,"col_char_2":"OR","col_vchar_52":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col_tmstmp":1375403282423,"col_dt":423099072000000,"col_booln":false,"col_dbl":3861.757,"col_tm":69763320} +{"col_int":-149645,"col_bgint":8720,"col_char_2":"MA","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882424,"col_dt":421244323200000,"col_booln":true,"col_dbl":27641.297,"col_tm":71068900} +{"col_int":-12159,"col_bgint":500752,"col_char_2":"RI","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1401668882424,"col_dt":421906147200000,"col_booln":true,"col_dbl":4024.414,"col_tm":5649490} +{"col_int":-2402,"col_bgint":5756000,"col_char_2":"MO","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882424,"col_dt":422671564800000,"col_booln":false,"col_dbl":1859.559,"col_tm":29932570} +{"col_int":-33399,"col_bgint":7486400,"col_char_2":"RI","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882424,"col_dt":421853702400000,"col_booln":true,"col_dbl":185.87,"col_tm":40411560} +{"col_int":1,"col_bgint":-1,"col_char_2":"RI","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882424,"col_dt":421853702400000,"col_booln":true,"col_dbl":185.87,"col_tm":40411560} +{"col_int":56214,"col_bgint":5372456,"col_char_2":"AZ","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882425,"col_dt":421991683200000,"col_booln":true,"col_dbl":10441.42,"col_tm":39161450} +{"col_int":-12209,"col_bgint":7899414,"col_char_2":"MD","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882426,"col_dt":422856806400000,"col_booln":false,"col_dbl":26268.761,"col_tm":28015290} +{"col_int":18763,"col_bgint":921520,"col_char_2":"NE","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882426,"col_dt":421187472000000,"col_booln":true,"col_dbl":4966.2418,"col_tm":39006130} +{"col_int":211066,"col_bgint":104832,"col_char_2":"NJ","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1404260882426,"col_dt":421838841600000,"col_booln":true,"col_dbl":1155.5,"col_tm":9018340} +{"col_int":73839,"col_bgint":103690,"col_char_2":"OR","col_vchar_52":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col_tmstmp":1401668882427,"col_dt":422474054400000,"col_booln":true,"col_dbl":8049.003,"col_tm":17273570} +{"col_int":124587,"col_bgint":617504,"col_char_2":"WY","col_vchar_52":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col_tmstmp":1401668882427,"col_dt":422182627200000,"col_booln":false,"col_dbl":5804.39,"col_tm":81757180} +{"col_int":-61299,"col_bgint":38096,"col_char_2":"CO","col_vchar_52":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col_tmstmp":1401668882427,"col_dt":421627161600000,"col_booln":false,"col_dbl":247.6481,"col_tm":27971300} +{"col_int":11345,"col_bgint":87088,"col_char_2":"MO","col_vchar_52":"AXXXXXXXXXXXXXXXXXXXXXXXXXCXXXXXXXXXXXXXXXXXXXXXXXXB","col_tmstmp":1401668882428,"col_dt":422137872000000,"col_booln":true,"col_dbl":9150.279,"col_tm":20510140} +{"col_int":-92531,"col_bgint":21100,"col_char_2":"CA","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882428,"col_dt":423093542400000,"col_booln":true,"col_dbl":9240.11,"col_tm":46147130} +{"col_int":95690,"col_bgint":186392,"col_char_2":"VT","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1406939282428,"col_dt":422953401600000,"col_booln":false,"col_dbl":12344.595,"col_tm":29546560} +{"col_int":-199048,"col_bgint":3520,"col_char_2":"SC","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882429,"col_dt":421109020800000,"col_booln":false,"col_dbl":19100.34,"col_tm":51408700} +{"col_int":216553,"col_bgint":86744,"col_char_2":"SC","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1409617682429,"col_dt":421599945600000,"col_booln":false,"col_dbl":1337.49,"col_tm":53136170} +{"col_int":-126232,"col_bgint":3160,"col_char_2":"WY","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1412209682429,"col_dt":422958240000000,"col_booln":true,"col_dbl":450.283,"col_tm":62477170} +{"col_int":120079,"col_bgint":6736,"col_char_2":"CO","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882429,"col_dt":422613849600000,"col_booln":true,"col_dbl":94963.943,"col_tm":16752380} +{"col_int":-179922,"col_bgint":198402,"col_char_2":"NE","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882429,"col_dt":423112896000000,"col_booln":false,"col_dbl":1145.5,"col_tm":25806550} +{"col_int":196285,"col_bgint":84560,"col_char_2":"IA","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1401668882430,"col_dt":421431984000000,"col_booln":false,"col_dbl":95854.701,"col_tm":7694500} +{"col_int":77907,"col_bgint":412832,"col_char_2":"FL","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1401668882430,"col_dt":422734550400000,"col_booln":false,"col_dbl":135568.86,"col_tm":21999400} +{"col_int":0,"col_bgint":0,"col_char_2":"FL","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1401668882430,"col_dt":422734550400000,"col_booln":false,"col_dbl":135568.86,"col_tm":21999400} +{"col_int":-185276,"col_bgint":163150,"col_char_2":"SD","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882430,"col_dt":421212009600000,"col_booln":true,"col_dbl":9255.77,"col_tm":37363300} +{"col_int":-14479,"col_bgint":5192,"col_char_2":"IN","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1401668882430,"col_dt":422682019200000,"col_booln":false,"col_dbl":1156.47,"col_tm":12482490} +{"col_int":-165881,"col_bgint":7104,"col_char_2":"SD","col_vchar_52":"HXXXXXXXXXXXXXXXXXXXXXXXXXIXXXXXXXXXXXXXXXXXXXXXXXXJ","col_tmstmp":1401668882430,"col_dt":422484336000000,"col_booln":false,"col_dbl":14879.08,"col_tm":21046300} +{"col_int":155858,"col_bgint":4640,"col_char_2":"RI","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":991441682430,"col_dt":422123356800000,"col_booln":true,"col_dbl":10227.57,"col_tm":40761560} +{"col_int":-8523,"col_bgint":696,"col_char_2":"MN","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1401668882430,"col_dt":421587244800000,"col_booln":true,"col_dbl":1026.051,"col_tm":17111380} +{"col_int":137899,"col_bgint":2799548,"col_char_2":"UT","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":959905682431,"col_dt":421273872000000,"col_booln":true,"col_dbl":1860.97,"col_tm":8261230} +{"col_int":1,"col_bgint":1,"col_char_2":"UT","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1306974482431,"col_dt":421273872000000,"col_booln":true,"col_dbl":1860.97,"col_tm":8261230} +{"col_int":-20449,"col_bgint":5172,"col_char_2":"NY","col_vchar_52":"DXXXXXXXXXXXXXXXXXXXXXXXXXEXXXXXXXXXXXXXXXXXXXXXXXXF","col_tmstmp":1338596882431,"col_dt":421681680000000,"col_booln":false,"col_dbl":438.068,"col_tm":18255480} +{"col_int":-86821,"col_bgint":11648,"col_char_2":"LA","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1401668882431,"col_dt":422381779200000,"col_booln":true,"col_dbl":1231.61,"col_tm":82189300} +{"col_int":65535,"col_bgint":65535,"col_char_2":"LA","col_vchar_52":"GXXXXXXXXXXXXXXXXXXXXXXXXXHXXXXXXXXXXXXXXXXXXXXXXXXI","col_tmstmp":1370132882431,"col_dt":422381779200000,"col_booln":true,"col_dbl":1231.61,"col_tm":82189300} \ No newline at end of file diff --git a/sql/src/test/resources/drill/window/datasources/t_alltype.csv b/sql/src/test/resources/drill/window/datasources/t_alltype.csv new file mode 100644 index 00000000000..adcf44d0cd3 --- /dev/null +++ b/sql/src/test/resources/drill/window/datasources/t_alltype.csv @@ -0,0 +1,145 @@ +1,592475043,616080519999272,ObHeWTDEcbGzssDwPwurfs,0sZxIfZ CGwTOaLWZ6nWkUNx,2016-02-24 05:14:12.307,1960-04-10,true,0.626179100469 +2,1194343172,3136812789494,bSrLdMAtGNmXsaFkmNpRaZnlPuUuSljiMqxJdgzoViIy,sluPVQz3991EyMTJL86YF8NEODzE1w2BtcKwPutL5TFUS7MM0c8pEYeWz2uq5pr0f0o0JcV5HRZs6gnwj3CPAdjisUk3LM5DvZcGUMkKpIEfziNrlcCrjRR10vWYVLeG6y5hG1ZsZgoSDWubDY2K1bZOaJTEP26AMDpYwIo,2016-02-24 05:14:12.308,1920-05-14,true,0.0794280235964 +3,-581053021,14567676734720,zBmUwoxbiWhazyIFwkyTsCo,b4iHRz7x3jgWYQASSBusBaOS5KjIKpdxRqEOkVvZSqgwzwS3r1 5C81WnF5RWzLNDOi8qgJK0JrMMjVxOpFi13socmJ94NVelUSvb6umYkzBrZhpzYDFHZmXtAfWzRBkXfESrfYF2ME 5lI2v79oP54eucAvtIQApA1Bak8Fw9VgRznhGwE nKzuew1UpXk6dAW91ag099LyD29bViFQxWzCyf7FFDEy,2016-02-24 05:14:12.309,1963-04-15,true,0.256111757801 +4,719544129,26552329517384,VlVwoLMKaardyJCdUttoBDIYVZ,TwDutiLBRsXHs0K0KvlzY6SHzNDJTpQFOigQ32YOYD7jCEpMJrT2SrvWBk9gJ7idoJhqIjW 4tPTI8u0cxFiEsU1IUNnONTYF5E3fGo8aXdA2Y0CAWqn4nHyY QPnVt3n5FnuQtQ88ayT3a0J86FOc641y4vCrTAqlN3jXVRJ1sg,2016-02-24 05:14:12.309,1995-04-23,true,0.367266831393 +5,-265354187,869394999808,gwfrW,h79IKUK0fji8POwoapUs7YCKsC5WDOI1Oge25fe7QXMAQrxIgyEIhj3upa62rTJxOVA2vCMg6U 4koGrPZzda1aNjvGiK5Qjc0qUPnGCjoGdgjE87rfON4Uu02J5 Q7NiZ15CtSu8IuY6oqYyZclq47L6FC2grvhcuBTRiw9C1zgQ6d1523I2RCONHyYiZi7RrNhziA2tSZCPmI42GYHtjZ,2016-02-24 05:14:12.309,1952-10-20,false,0.886914049583 +6,-739409800,458112011400,upOMZlypwcASQHtEkgpYybPwvjCtFHpuJkfNlQBSIrWqznQJEzEV,tmZET5agjU4yZaPXZDGW8iSPwRmkuGq4dpdkZqtv7edggkdZm3Im1j9okgoqqcuqPmo8r2EDK0bPZdk5YFXutZoAVhK5ecd06gGdNj2tWkSVKO7wqCUR2E7obM9ZYpZlgDecXTjzLvLUYuZy0TV sTjyt8DCU1QnKChPKD MbcopdpZkaJ5vWd5tIM8eYOB5kjdTiLfV3TVYb2LmZGt2f4xHbih odqbeq2ZDeEuFAwaoyL3UQI7Euj8,2016-02-24 05:14:12.310,1969-09-08,true,0.492183947524 +7,2022947631,122811962676,ZAFOcferhjkcl,1wnKPdZ2a6nHOkC5kuo6lpCUtShnNjjjUd4f3vBs7AdhwQDoBPA CRMC7CCBTU VOx38gwZurb9HrSQNRrSd2kDuHB6WpqffMTwNfHPvqxnwHtbh4M8GwVEE0ocFsvYPkd4mIl7 uhdOOOS3H8jZpGa6Yt OoalDERUEcgZz,2016-02-24 05:14:12.311,1921-01-25,false,0.60032636853 +8,,,,,,,, +9,358750156,128037619138,rNMsKIDnkuQFIYoJorACrHkFprLYMpp,fEwIVF31I9jk4C04qBYtw6pxswxIuThE3id0UuKTp6CgcnP9eNlVwJh82xvwsN 8CmX1RSQhtozud6Umv6kLCKSVF7BQbxRS07V78HgbrH0ZDV,2016-02-24 05:14:12.311,1946-01-17,false,0.361570558696 +10,-116481491,362751773072,IcStZtjaSEfWDmrulvsnCwUQvfgn,aTISWde3pj5OJd9l7Zo7j9Kn8RRdWimrrPlc5BDFN3xpqrliH3Che9OJERMsQgwiafBwavLrTe21ORPrcml4Bh44fDbC4fBVvMMLTPVWbmgvmXm02Ct7A1WBMTM2C74LLJmIdZwPpwMXumO5dYZg8Z3KU2FnJ2bl80qqgyjJNlv9,2016-02-24 05:14:12.312,2009-11-08,true,0.488525932471 +11,993964305,713109032528,TuFaJOrTexEQKCXg,9VLdcEruZzhtTbrz6Tu5vvfwwIy7jVtc7R5XEeZY,2016-02-24 05:19:58.301,1943-10-12,true,0.957695228383 +12,-1735090787,841324401984,dPRmWDinhineYImsqZzRZSSMx,srBmkYpJY7TC0W,2016-02-24 05:19:58.302,1937-03-27,false,0.707961195747 +13,-101621245,4569922715352,KkQGklqtAFOBwaNXDWkN,WMhbfG37BBZSwtryJVwMlS67TYe5UjqLYa5pHES13GC,2016-02-24 05:19:58.302,1953-01-05,true,0.156305915074 +14,202631122,6199873199432,qV,w5Cs5Jas0T,2016-02-24 05:19:58.303,1963-11-25,false,0.709658352873 +15,-1800226884,2408646998880,AtfZqknYOrYBXcFCORVnzcKmXyaEe,xfxBmCGvTcaBxEFXSvPQTaCZRCgE9Cus,2016-02-24 05:19:58.303,1953-08-07,false,0.22155866245 +16,2042714193,2437409008036,BQyDpxxWldiVnQlFFfasgfwIIzpgJd,YF6qr4rAyBrHa94CosX6BWpfbwE8EXWXrupuFPoekm G1PE6vTKeliuW gxuq,2016-02-24 05:19:58.304,1939-09-08,false,0.305618728365 +17,-1236268475,090275297566,OGnxbGFEaWMZhrJ,Tc0A7vQFGyzGXnoa90wXRVEMaZMmSTix2IqRw48WKqE50hCw,2016-02-24 05:19:58.305,2013-01-18,true,0.141061129745 +18,-1451268845,7456257725280,HAUtIjkTOkHLzjhucQeLuH,a,2016-02-24 05:19:58.305,1932-02-01,false,0.555072055925 +19,-148845,3757280,HALzjhucQeLuH,,2016-02-24 05:19:58.305,1912-02-01,false,0.455072055925 +20,533611126,784427291198608,JgPGWgjgNEjMXYYmWtC,PY250MG5pFd3NgQJGckqfC9mwY0DNazIAFmqRy1J,2016-02-24 05:19:58.305,1929-08-13,false,0.355503179385 +21,-642124822,43598566290864,syewpw,iV24XUsay9jBU2wZgUoCErzIYo,2016-02-24 05:19:58.306,1956-04-02,true,0.456846886729 +22,735561305,6307081519934,tMi,ig9ClFlYmzIAb99UFoGaUyjSorP9E9qA34km539T4so3qm9kRZZTZ,2016-02-24 05:19:58.306,2002-03-20,true,0.740283135928 +23,-350986527,69653384392040,DkPvPKYuCdSatlPrzU,zfSgmg7tAawKVSJhkipg2nSNuprAp75eKDJry0focwPij J4hE,2016-02-24 05:19:58.311,1930-06-09,false,0.336419256965 +24,1847465545,38629268606556,PV,D9lY2D4VJGmFffgSV9MoyojaAjNVxcVyBiEdHN911nwK EBRi2wwFuWwbnyLl,2016-02-24 05:19:58.312,1923-09-02,false,0.964243969107 +25,1442790109,61368071783056,myUrrzUHGnxasz,wyweWUTyBPuo3jOBJLPfSWqji,2016-02-24 05:19:58.313,2005-08-10,false,0.507695339931 +26,-2125195491,57004017502044,iqwqmwkUhrtTliZiPXRTstVpqgm,yCko0ouh0ueXmXK0O9 MPvLcCmlp7d7AI4ca1ru8d4jGGtbPc7mPx,2016-02-24 05:19:58.314,1922-10-09,false,0.962960554859 +27,-603256047,64691852740816,lXEgAlS,7ceX9a22fDB1wb9RUb6TQWqvkbE6sXp2Sq0Lt,2016-02-24 05:19:58.314,1942-11-05,false,0.940132365478 +28,-1806928495,73151265354384,yfjoc,sqxWGArGIX5jKOvvOI6BjG2l,2016-02-24 05:19:58.315,2010-05-08,true,0.769634808418 +29,-926147543,18430182,kyTyhomxLi,9TR790gkKg3KMkc5k47V9QAuos2Ojig04QwTdO5nB,2016-02-24 05:19:58.315,1969-11-06,false,0.754223352027 +30,626932646,3285206109420,IWZriOkCaYLDGOzMSp,oSv1RhKIOtLgOLe,2016-02-24 05:19:58.316,1988-05-22,false,0.444467141108 +31,-430498956,1258459028142,V,Q1Am3KtwYAQx Vj7kk9nsawCmN BRciIQiN3hF,2016-02-24 05:28:21.220,1924-03-06,true,0.210929121391 +32,-938167176,6017016766800,PStTin,dfOY 50EejHegPw9axCPz,2016-02-24 05:28:21.221,1921-04-27,false,0.827238030237 +33,-464056232,269431279554,HHqSmnhL,VVRz QWFjn96AtBYO7eR001KZwKGLUrEqeMrPVcKSbQowuQ90XXFrl3iG,2016-02-24 05:28:21.221,1991-01-17,true,0.764629411376 +34,-1564810747,471032170712,XJF,8D9e1bT5wwFpLD1Hr92VKaCQJS,2016-02-24 05:28:21.222,1959-03-16,true,0.804630256964 +35,2047004122,6190065899528,PAobaowvKNSaPsKEdhLtCRWe,wGlZo1sVGPg,2016-02-24 05:28:21.222,2009-08-09,false,0.119950105444 +36,-682355573,879651728,BANiwCtAOccuLIqNMVOQGBIQKNw,KxgxdOznNSNaltnFxkxj94qZ7C05fwQVUyBkN,2016-02-24 05:28:21.222,2014-07-10,true,0.152121113454 +37,1168899469,311335712,bQoLkPBFAUXqcMpcp,6YE84Kb8G6Z8 HR9qkTp30a7cnBh3T rC1uWP4o3S7DSm,2016-02-24 05:28:21.222,1986-04-03,false,0.674319849007 +38,-1678398290,895040440,CSavGyXMbAANckDiCshpuNzCmBXuHbHw,V VabLjrkDF4IJ7twnT9gqTv,2016-02-24 05:28:21.223,1930-02-26,false,0.249230799416 +39,-1433668371,102790112,yytBlhCMRSmV,fanhgAi1Cla 3lwYXQTvWYdIvTF5ziIBc,2016-02-24 05:28:21.223,1976-03-19,false,0.538207272457 +40,1300934512,1526006148,WYqpkjPLSeCA,4wK m9vhE XTHlpHv1U,2016-02-24 05:28:21.224,1934-11-26,true,0.0125838026238 +41,67513000,8652621344,cTPjdg,KvYiWTtwEwEBTWUArtiUnPciZY V8VzRIVI03Gyt,2016-02-24 05:28:21.225,1967-08-25,true,0.0616766496679 +42,1071705345,774763920,YfwqGtgVkBewvkIlQnEEnt,EtNWvXm45 Tw5CI9YjwflvI4zVqHUUpLal rAt2f8X0Wfp,2016-02-24 05:28:21.225,1978-03-13,false,0.881191135812 +43,-705176971,1305480928,xLYbWdnPrlTIt,mGVBbGVhQkAhjQdAhiWAwHlCNN5f1wggwKxRjsBBbmlvodajek4E1r,2016-02-24 05:28:21.225,1922-07-05,false,0.0963362944467 +44,-790060808,2091106612,chvsFysOhXbqRnniYOGeLUppugM,XZkwRGa0gxPwmnAG4VOQ1bNNgzM9cO1RzrcynKKBssqEP3BeEhibq,2016-02-24 05:28:21.226,1927-04-20,false,0.292481581027 +45,1221413457,8339791280,SeQaEfXkDgumVjpOKOglGiXVe,CwZrVQVWdAfMtFcqWFJzjTcrnpWT48Rbkd7CO,2016-02-24 05:28:21.227,1971-02-08,false,0.648820911939 +46,627701102,33308856,qaOEIqTfNGD,OoDOYgbcJ09XmxqPv1e,2016-02-24 05:28:21.227,1992-08-05,true,0.40799240898 +47,-841849915,42746339248,YPiESbIjBms,xFfL8aYVA5EiiSWmvKkHAYRGYCiqctdxQwdkH4DOP,2016-02-24 05:28:21.228,1933-05-16,false,0.324089746843 +48,735186776,61478943200,BJIPsOpsMTNRCO,80Z7O0f6A6fj1Beww7oOkwLSZdjO,2016-02-24 05:28:21.228,1957-02-24,false,0.359520992037 +49,56927672,84551700064,Y,Wvu9T8ilDl8k3oaB0PFNjHirDoMdwv1YYCah6ahqiuATp1,2016-02-24 05:28:21.229,1972-07-10,true,0.328379339497 +50,1182933670,45692038432,whFjvGloFQxwYmvLkshjwnADhUQlV,yt49EB4Y2RAnPQXNwTqfp7qzgDH1Jf1rjY0FnpIp2y4,2016-02-24 05:35:05.544,2010-07-08,true,0.835195212647 +51,980460718,5971386168,OMJEMdywCTpQgqJUYDuKDxAPBkWdEP,akbcHSDflWlG8K6MFnIOOLBMZ4z8NV2FecgqpLO8,2016-02-24 05:35:05.544,1939-06-05,true,0.0594494786115 +52,-1263020365,17414114976,vkUYqgXGGfELNjVwyYKVJMd,7zP0naU,2016-02-24 05:35:05.545,1991-03-22,false,0.128559830476 +53,-2078473628,23813625600,PVAKCnVrihXStlfqJf,HKyyDE7cHRTzEQX2F,2016-02-24 05:35:05.545,1971-10-20,true,0.497832439707 +54,-388853860,4882213296,NkSynIivNZiIplsAegulLrz,2AWYA664YJkqeGXGZeI8vf3fhX06wjhlUoRdzRBJnylTOnUsrz3LcyxbTyIQHr1YCMXl9jFx6LyhnaOYZorN4rOazR44zI2,2016-02-24 05:35:05.546,1983-04-21,false,0.806218120908 +55,147766593,3544613368,ByZRmalvDsZD,XgulsUB2lwxhZaMz0POOTcxIAt7mPaBialPFYzth6Aa54OPBoTY9nlJpXWf89oIczO0QVg5lbkmQQUlljIMEe2Mh8rjtDoYox2uDykAXeH0BXGVtOZ12EIwe,2016-02-24 05:35:05.546,1952-08-07,true,0.189172246469 +56,420230528,7209802880,ZwiFgCchxUqJTEShealG,K7RQD052LFrkPBfuL9hIihch,2016-02-24 05:35:05.546,2006-06-09,true,0.462810537514 +57,-1894216997,5384805744,eBKZaQYxrdPMRRwgeqPtJjxjqOBqUH,z7cJERSftFvbtxxgVbdF31jZCHmkHBmazWaHAa3eKAp1PPEObxkcuhjA1vf6cSlGe4TT8Yektdwqcr1Jd,2016-02-24 05:35:05.546,1931-02-08,true,0.604145500315 +58,-263032861,56565941904,dqXVKoNjMFKcIszKFRXCotmbejjxGU,3QDSogOdQYe6NfDPkEPIHhtECUKeT3eXsRdbwCj3omYIHz3HX,2016-02-24 05:35:05.547,1932-03-23,false,0.472430725946 +59,-1661674051,9595495792,atHZ,7SWlwF1qaCNs3Jea1NollJfLsgTK95yuo,2016-02-24 05:35:05.547,1974-05-20,true,0.0923867105652 +60,-1129907976,6907332688,Mc,4FN,2016-02-24 05:35:05.547,1963-11-25,false,0.701638057307 +61,-514254242,195633336,kenCPo,eqVUbd9TzGrQS2W1HuV1apKCt1rkYLRTHgED2RIk1jhowGYzlRa2DVMpzTVc8RVO44rm9stOD9uvJPLNE48jVMQfhLwtNDlKXUchY8N0FOEsMW2G37X7dYnzod,2016-02-24 05:35:05.547,1992-02-22,true,0.511369563541 +62,,,,,,,, +63,-535361692,19818839912000624,aAbGSv,yNVubV8dlc8KC8aijpwxEFyYSqtI8iiAQeZJAQMKUqTKMD5EqFSG4z3AraZ9dX,2016-02-24 05:35:05.548,2008-11-11,true,0.587212121798 +64,-960817944,295389276379024,ToiqQVHAUTzNXZxhxT,pf7gyDzSkNhMD1Cio9GnoDpe7tFF6lPUfskrdSBrdmkPrwPN5YrF8cYArRXsqeiWiUwhbQ,2016-02-24 05:35:05.548,1984-10-16,false,0.425544340933 +65,2079413364,875274205296,QiyvPpHnbIkkLsJeF,cuX2mutS6Dt23LpVfQ4RhuZ2upAZZxSDl3NQjHYecHX6UpRud5J5GUO2yKikEShe1AeqzPbW2W3GVn3pwpt5k9SFo5BbAQSgBpXWe7OUYuLB6XE0VPbSk6RK,2016-02-24 05:35:05.549,1933-03-13,false,0.201374181269 +66,-786819448,64868238648,h,ScG74UmrI77csOCHrVQQMGnAtWw0IOpLtDTp6xF8eH2MxrvjwW5BhxTKxSelvvDK4LJlO0YhFW6CoZM62aUEkOmtXnTBp7KH4lAiCPW2lnC5DEHn,2016-02-24 05:35:05.549,1930-08-13,true,0.795415967241 +67,1250142164,6916992,vpZxaEsZkTwnMmmWa,dQxtZCEAviuZYSdc4sgLjb,2016-02-24 05:35:05.549,1994-06-27,false,0.556953353203 +68,-38579995,233512622900,JgwMVLyvDb,fwuYfa4PJlIQoxfetQdAaVY4VvbbX0N9E641,2016-02-24 05:35:05.550,2012-06-26,false,0.453069633364 +69,-1027828328,40672068432544096,TZGIkEEfJb,RftjjeGNAAGDTikjbjJ6U4GxR8ZRGclneqPib9RyyXI37Rr8E3DstJ55jBpN4KY1j2Dp44l6NkRvDHQO,2016-02-24 05:35:05.550,2007-10-03,false,0.998189105778 +70,-569066973,8907959568528,eah,2svWW0,2016-02-24 05:35:48.510,2012-07-26,false,0.264360661376 +71,815270244,15558122158,l,5rZomJ4gkusDD,2016-02-24 05:35:48.510,1934-01-24,true,0.85139020182 +72,-2085514660,1345100636752,Gb,VecY,2016-02-24 05:35:48.511,1956-06-02,false,0.159955514577 +73,-2086451715,778834561344,LGGRD,Dv1,2016-02-24 05:35:48.511,2007-10-17,false,0.892355433411 +74,1317696149,13607285678,Zf,h6rrw,2016-02-24 05:35:48.512,1930-06-25,true,0.226484732 +75,-475078365,591562831712,Gas,Az81IOQGM1,2016-02-24 05:35:48.512,1928-04-26,true,0.519357968945 +76,-1610419012,987957004000,u,RUUA2mTktOtMRpv,2016-02-24 05:35:48.512,1959-11-06,true,0.283395173271 +77,1,1,gn,iyZ,2016-02-24 06:35:48.512,1931-09-24,false,0.714975974474 +78,841562550,5229353983640,kfJpOk,BP4z,2016-02-24 05:35:48.512,1931-01-26,true,0.773946465042 +79,2049749227,7886072056192,lJroaw,CS,2016-02-24 05:35:48.512,1937-09-07,true,0.866341378246 +80,-514969856,15673249928080,BNnLZdPW,l5a0ZpK,2016-02-24 05:35:48.513,1960-07-13,false,0.500069917482 +81,-889738985,32298052657224,xFDhe,yqzQp2PYA2w0r,2016-02-24 05:35:48.513,1969-10-07,true,0.534138245169 +82,743747735,42939563416968,kcRx,ALGpDH9qQ,2016-02-24 05:35:48.513,1945-06-05,true,0.0339015122383 +83,-991380116,7521740669440,I,VnrhwXNs5I7A,2016-02-24 05:35:48.513,1931-02-03,false,0.729684203442 +84,-605301438,6462434675584,OXqgNP,XvH,2016-02-24 05:35:48.513,2002-02-01,false,0.741169865031 +85,-9600122,1150794800496,XC,EZ5w,2016-02-24 05:35:48.545,1969-01-25,false,0.00508533320078 +86,-1304870155,7762439826812,fOJJ,3kmqcAhqRv1JKH,2016-02-24 05:35:48.546,1969-02-09,false,0.857763501506 +87,-1151111063,3026615127030,B,UsA2ijb5Q472,2016-02-24 05:35:48.546,1988-08-16,true,0.852362483167 +88,726080083,352777524,pvjG,6DTPUc0kPXNDI,2016-02-24 05:35:48.546,1984-06-17,true,0.499536294849 +89,1269715757,298591968,wUJle,P9hocYIzILG4d,2016-02-24 05:35:48.547,1967-10-23,false,0.731465721495 +90,469877353,5604128205368,cgBBlTOr,0S09,2016-02-24 05:35:48.547,1926-06-03,false,0.930318601251 +91,1439825179,8448247315360,GhHH,xU,2016-02-24 05:35:48.547,1995-03-19,true,0.365706281943 +92,-965564746,6461158733064,ApKK,2MT6vQK7ILiMym,2016-02-24 05:35:48.548,1989-10-03,true,0.160958193689 +93,1396621364,5965143616184,zavaYRc,m9Y,2016-02-24 05:35:48.548,1953-04-01,true,0.123678441424 +94,811012151,1275108040346,yQmKx,ZHAGJpfsrd,2016-02-24 05:35:48.548,1956-10-17,false,0.784623709976 +95,1117245576,7044500976200,fePHGR,pjLll4ngRT36ygIV,2016-02-24 05:35:48.548,1955-09-10,false,0.187592141607 +96,-450291430,7186198454512,KhTYcGy,Y1l,2016-02-24 05:35:48.548,1923-10-13,true,0.348668175704 +97,0,2668027779964,kdhbgC,ejaUw7d3DEIC7axy,2016-02-24 05:35:48.549,1936-05-25,false,0.690933751147 +98,-1807107154,667184558696,C,M4GVt6c4s,2016-02-24 05:35:48.549,1935-10-08,false,0.868842793576 +99,-1610548040,269450722304,UECld,DzTXQDe,2016-02-24 05:35:48.549,1973-01-08,false,0.866249476344 +100,1968262238,1853568964,XY,O61h,2016-02-24 05:35:48.549,1963-06-21,false,0.771913788526 +101,547475512,5424751352,Myf,u91,2016-02-24 05:35:48.549,1967-04-02,false,0.020687569041 +102,1347375329,7601674368,PubTJ,B72D746BYbWS,2016-02-24 05:35:48.550,1928-09-20,true,0.669628865357 +103,-730200083,3734160392,R,JF5b,2016-02-24 05:35:48.550,1957-07-09,false,0.754789130107 +104,-2145249349,8153964480,BAGi,o69DyecndyD47,2016-02-24 05:35:48.550,1967-11-23,true,0.669354481782 +105,1922528937,36022570792,lJy,8iirhhzlmMQeL,2016-02-24 05:35:48.550,1979-01-27,false,0.53821290676 +106,353908213,108831386830,Z,vb32KaNk,2016-02-24 05:35:48.550,2007-08-26,true,0.772742852356 +107,-1,,kYEB,UY1wFk3nA6uIg,2016-02-24 05:35:48.550,2014-02-04,false,0.00471118728446 +108,465659139,196283469856,RVVu,crs1mym,2016-02-24 05:35:48.550,1983-09-03,true,0.510821715226 +109,-1330396672,584831936,,XxXpO,2016-02-24 05:38:22.869,1920-05-03,false,0.0730700338964 +110,-220130197,17601020,hjeac,n,2016-02-24 05:38:22.869,1988-09-16,true,0.96826163121 +111,-42902,562336,f,LykOIn,2016-02-24 05:38:22.870,1935-01-19,true,0.554463637243 +112,765525961,28891422336265300,demso,LZoFRza,2016-02-24 05:38:22.870,1943-05-26,true,0.850373501413 +113,-68207234,90427790806585824,pnmrt,HFo,,1942-07-27,true,0.71373237138 +114,-1380251111,63163365737847936,krg,JXzNCV,2016-02-24 05:38:22.871,1966-07-19,true,0.445900639514 +115,-1844719171,60499043191834664,tdqfhi,SzMEux,2016-02-24 05:38:22.871,1984-08-13,true,0.520455630248 +116,1790522162,47372667161181496,orbowc,n,2016-02-24 05:38:22.871,1958-04-24,true,0.891198277906 +117,-27319471,37102817894137256,w,SYvkQ,2016-02-24 05:38:22.871,1944-03-06,false,0.0359150523033 +118,1317690445,61958708627376736,aehmn,Vioq,2016-02-24 05:38:22.871,1935-05-16,false,0.943878204382 +119,1965293154,4216354032582922,d,RSlQvgb,2016-02-24 05:38:22.872,1943-09-13,true,0.300925886142 +120,-1998776606,80843181252171408,hntp,lt,2016-02-24 05:38:22.872,1984-11-10,true,0.81436509096 +121,-759066837,56811856992247296,gbfr,RXwJTUQL,2016-02-24 05:38:22.872,,true,0.602622967887 +122,,16079752775114834,i,wOKgo,2016-02-24 05:38:22.872,1969-03-03,false,0.314540418049 +123,-1393583038,21198598572581100,cg,jJo,2016-02-24 05:38:22.872,1974-10-17,true,0.863833034127 +124,-359360839,10812882337641318,yn,pp,2016-02-24 05:38:22.872,1987-04-17,true,0.984219215225 +125,-135559442,26079995480527800,u,PpZ,2016-02-24 05:38:22.872,1957-06-27,true,0.112619968355 +126,-963314220,39899141337524696,r,Kv,2016-02-24 05:38:22.872,1949-09-16,false,0.928182322637 +127,-603033839,91979853046157776,iknfwt,UxrG,2016-02-24 05:38:22.873,2007-09-17,true,0.436065582677 +128,1730933740,52988296769210224,,SQreQ,2016-02-24 05:38:58.864,1920-02-25,,0.709400581459 +129,-1190592568,29537626363643852,,JNq,2016-02-24 05:38:58.864,1947-06-02,false,0.426506472311 +130,-1311152842,24893851443643300,,VZmvS,2016-02-24 05:38:58.865,1955-09-24,true,0.891508292788 +131,-612732312,14972217694439486,,yZYlsOH,2016-02-24 05:38:58.865,1939-09-17,true,0.682577092573 +132,-1948510563,2686369516214692,,HcwNvv,2016-02-24 05:38:58.865,1921-02-06,true,0.496484780895 +133,1726823975,64270113732818952,,UPYQXFGA,2016-02-24 05:38:58.865,1969-01-19,true,0.858728636519 +134,220535371,15937600070688400,,tTsISx,2016-02-24 05:38:58.866,1960-02-18,,0.0978174122941 +135,-278647794,52598911986023288,,JzQY,2016-02-24 05:38:58.866,1975-11-10,false,0.0918752610245 +136,288414270,8695637273720187,,OpFxaJUK,2016-02-24 05:38:58.866,1935-01-04,true,0.0241314164497 +137,-1033256154,2397384354833930,,,2016-02-24 05:39:29.484,1965-01-27,true,0.817785665204 +138,-461761396,78042258136748336,,,2016-02-24 05:39:29.484,1966-06-15,true,0.514550507111 +139,-1863053037,35173898947893320,,,2016-02-24 05:39:29.485,1937-08-24,true,0.641830766022 +140,-775202711,21011901540311080,,,2016-02-24 05:39:29.485,1937-03-14,false,0.563296650508 +141,-109815917,46096729108954488,,,2016-02-24 05:39:29.485,1969-03-08,,0.82789510574 +142,1733733471,84276421148786976,,,2016-02-24 05:39:29.485,1937-06-19,true,0.737905542986 +143,1559768019,91381489058732016,,,2016-02-24 05:39:29.486,1996-08-19,true, +144,-1054306811,,,,2016-02-24 05:39:29.486,1968-06-16,false,0.273335960277 +145,1580853588,17990322900862228,,,2016-02-24 05:39:29.486,1985-06-24,false,0.668501595682 diff --git a/sql/src/test/resources/drill/window/datasources/t_alltype.parquet b/sql/src/test/resources/drill/window/datasources/t_alltype.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ac7dbd5115c636fcfd3c59f27bd76bcc4cecfca5 GIT binary patch literal 12340 zcmbuGdA!q9+V~T=;05o+3vw?PM64i)AliKyk&y09x3o={WE4%)HA&kvO`EpkD2gJk zpo6=kh$|z5xD6=i;4+Mk%is!%3#cH1xD7gtGwS;UXYu#`-uI8+=f~&sy-Ch_p65Jg zd(O#eAn~|i5E;CF%;0TfP90o50`5l-_D&t{xduUy;T{zJ4gtsuJ@%1sSv>;Pl12dN zF^6m;df;XhFd8@t=mo|AV}YZAKA;~M2OI+o0ONrPz_GwYU=YyCm;?+1lYuF~allmI zcwicE0-ys<1f~NsfRlikz%1ZoU^Z|HFb9|moC?eXP6JK{^nd{{0w%x=SO65Fu06!1_2p|ZAfG`jNq5uiRfH;rfHaT+ zXn+BJ4x9y?4J-oA0nPYZ~?FwxDdDqSOQ!OTmmcwE(MkWmjRapR{&Q6R{>W8%YhZZH2@5= z6=Mc({y&GDLVfv*;hs&CrY{~E4KmO59P*9mgPV~T@aV|zOMni&vq8@2IYaNe^Nq9L z#n1V0j-}`HFWQR_zIOViU$tMHlpUP6dfeZ}Ke*tGU;ldLjU%&P{_WL|kL}yM;h~kW z%u^lDSn_wr<}X@9lh_PM(kO*{}7npS=HXFnv5pF~|a{^+CM zxuzLfdCJrmpS$ObMKhJ-(tF#FOnfdgkp5`Y2am6Q`=t@xD<)FgulnwO&$_i=-*WKK z6~B=lxIBE{oY6h#!ChGM?Z3Wpc-6jbG2$hE_3QG?KX9+UlsV?rxf}1!jl5_2Mc#FH zNk4n|%?*#_MjU_ft;=8Chd=Rh?_x{e|Kf>v58ZnB@I9LsO`&(^j+(S< z-^5K@wYFI{H071o?_EOdAs_zk#_KjeJ>&6-@8!nas6Xr42bX;3xpT>N^5u~kyVrfV z`Ot_}D74$}9H$r$tUtAjoF3UdY0Y2%bi;b;RPl*@&pO{Yuyg5>$`4;V9S@ZyZc}24 zZ|=Orx?uTfbn0>6y^{RhJ@QubmIVphKw7_S+YQbslTXbI z-J#n}M#+Ombzj$dY)$J1`D@GY;1BR`Wd5n+kf$GBFHGG16`%Rxhv$D-HShd0KmYsW zv73JT#4AT%YcxH^$7Y=KefX@)BiZj~G(Q-6ciG(KPrd5A_4BolUB7bwXG;(K_4V&o zZXI76_s*w(nOl7Bhu+7m`L`Y{$=#j59(w1BlNTH;1gFgGA9rf*^966~$Ip7E{PdL* zyB{4p{?LkY=7m>&l~(?C&e~M*wYi7S8g=#Tr>IXHi(tSV8bA2t_@BnO-fiQfoc@nq zbhuY*F74k?FZ}to_mBr22RAxSjP!tA7Us zy=}c8FL)CI+=_=z-+8GMLH61wAF2MQeOK)H2b8z&ZDhnuNFnc-`BfY2J$=m2H)-*2 z$FfK6*J4TUj^EC|1JYLrRV{oXx?8T+!a1_>4BDbycQtg{^hGX4zh=-ieY&NaN1k|k z--@s2orVtd?Hac&jP~}KpStrrxU4^Z(wFFky&D()13gNQtUqV>U&d$#998nfrJ-}L zUi-kY@Ti;DymKsE?I(Q7p{ES2ihR@GgW|I%6|Vp%ko9Yj8R&4|;S0U5p_BR+zq9$b zP>S^HuNR`dr>@v~!(8YfM7sO)Gtd=%D$PIEw8Eey;~Jqlv$9v(Lr9iD%#`q3sW4DMR`+UUJ-MbAUgFXA^Ydj6`XpraR$ zSvu|1qo4tQb-opcapqrj=x^u12wC=P@$P=;NW{60*bYs&_=*$m-34`&j^FUb(NM^F z2i4>;p!dxSvGrS^qs(8A#!nwWcFlTmJbLaIZu^`G+lE1W)uum=cmTwDEfapOsc)R) zeout!&9|NP!E7+&m%lx=1m?du_vJa4!S%f3-ZECeTuBb4Z$SgIH!s<&b@#KjUsTa! z`VLP|yU=I*7H5Y403Fu*?6vPf6K=VEtceCp2B_TSyAD*-P z^EVOuFU{BM!}lTgefQ+yZQGV0=Z@`;n0CTD$XN6&am%;kkQJ%EgLUf(C~{l>8G~;= zkHlBa*Ee(y=@apQ1rXgczy>of#;$B4m%ApcY0 zeqzUQo1VQB(LJ?<{kk+4{UV*)|Lr|)bmAp@n^X2<+S>Qo)WO5Y{WKIE)n!{O9p{F7 za48rZnggx4Vcel>d;SxJA`pd!(36hrl%iRGK25njT-MiZG~6nwHswO&R8Yz$6a!-kDXIlItxf-?6b&UHAZ;4o|HBx8QdG?3VFH23!HcYB*>9)BN%X3g=Wkf zpld`;#$B>s@Je!s4oi|2NdsI$VHBQCcnWEEMQ)0%M%ASzW3585Iso}p)5{W3naa04 zv7jqcoD69>GG0l$%hgi3nx_MmMxE)#nM5Moo(P#iA?6drx&pp9RW_1wr-ca21f-tQ z0yeG`n7UAJ$;I-areA9kKmL;hE%FgX(w8ps z#fXOwXK1>TR$MS&kwz!4O6o|sQ^I8}^tuJIMF#1nE0xSL+p3NS0)U63a}P%*3gNl9KAh3PcS=%y^2pn9w@8Q{Zcy7J6Y+bV{{K zJ9MPDK$?-uQNhL1A-6Bu4JRa+-AGDvjZ7$1xvIqkzrUWSR7)Yv%?#ny)<+VrW!akA z3Putv*$8@arI6x^r}LUqPCQapT?!uMxMnC&kAzdvn9muA1taQs$j-)UA%S#y@nojX zg(XhXL82?cwtSsTIE&*EPH>l9jgr)ibX^r^rs>c6Rc-Is=_pRIX}rq7U9F*AQxzKgWh!@ChX?SiFVRAO z|HVS8R<6fVFsZ0+B*U~zZAu$2v2v$HO8#m#APAubk@sp%S<-S^64Pcq4Mw2UYPeEK zv-k%m5k)ne3UylUEIMu>#rOh($5SA(+R)9jCG<>0yl~2FUsjtEA)hbqMvv)X3h0Tw zKDL3Lh&0g&J;f?}=R~1ENKpYDJ5eO$8mqDsraB|l5R)U*C_d2kx=VTV{XT(G#FaX~V%1MRUFmIyh0tm9j+_nnzsdyn&?0 zCFVw5h|uU;z2Qy)9qJ>SS!&c}NF2@eIdNYZJ-$~YI;b6C(22bQ?HP4*A_~ zI$??k*{!tK9`QuoU5`OEIu$Ocl1(LKp}fh2EgaT!cF9BM(_Le;ZjoetL0{7oT*~5) zGYxC8s`4fmiRbuoBPp0dVT-%PaK2<%2ucA@p<9l~BIlC%c-)|GrRh{C@3VF+C7*$5 zFvVKJoo;5`v@vMNGEo*L8fmoRVYfu9f!Z!`HES;W<6T?Z#23?a3{NDS&4!bWCM-cd z5R`K5c-<2fQY|J?FREQNVUi6xi_30ESG|^ax*LkPqh))&NCfqS91ioTc2x3~4W>qo zs}K%zB$XGFiB{Iytk6Q&DKjOhOS`>(rjk+cLb@Aw3Ymf@As1+`G3?P<#DKA7bJR$S zIp5^)mJ*2J5^iv^g59e+QpI?;TJotLT{YNk=2b&d$_UmRo^}}6qTb;Mb=*crHdTgj zm+rbc1)JCF_H@=l9Vyk_RLUV|ypi!Y^g(^FC3b0R!rzU!2?%}OXt@|O8R;nPw(%~H z6eeWKn2=j(r{E|OwsNjUl3Y>%^>=7pjI5>*247b7MBV8dleyQ2X^w3s(q3VB;B6UJk5J5cO+0wvzm zlvt}#rigYlg&SM0aM7C7nT489CYf%IHLwv_4az}FjOX8OH5P+sg@3 zjaP)GuWrssIf1DwEn7ZUEDAZMVhU6YyrU)*YxO$Utdo_RT{e2$L4B5Fa)O2SW||69 z!&9Z8CC}yb)}k*L;f+dKNTh;Mn_71z<3^9ImJK@?T4KbaJI`e(z8h+Vl5~^l=&6*h zLGc~M?sg>&v1-s&B7<&SILpZA{L3u ziJVbzH~mg)TB;WcVM>h@{A8=%uBv_|TQZ03J~id>=xdy}(V|HqXRZW9o32^PMWfM# z$#3Tw$;Vo0MHj8HqT3TsdblFft%9!wj~YmMO@@Gj7tAibx$dMD)he|cYCF)RN*+@= zr%yJ6lw@@kxhm<%M_}VmlLqNF*oL0c z!Be}?c2-=z7R9m7w#S=Ew8$96x|q1D=yAB3jWa=p_FC(*S;jj~+0Q!6u395wb*804 zR`u9DnRG^ON=+|W%L^5q&p>*tEh?B7%gwZ|*lyPKV1VNExUCiQ`Z`@-rdneidNmVI zdBQ9ct|X{(+Zd?FRkJ-Eta~i_l1tSVV^(Vzk~UiOVV_0v=1h@djPP+nz2I_GR9f$j zu-yRet<>Wk1EIYxmQzM^CfQ&+4&IqGs!TCib9v-0yvb>YLgk{vC3;1=0!mqjBk8UA z*kp%K)MV1(anx~>S*Tb{4$9o9nd%jX5N8;QF@oa%`=YNkncE@KMz=Xd_$uB)p_oWH z!nIC~XK_A~YICWMv+T+FBMyV1ss?o)XS{4wRc|w`W~-f$sBPiZj+##=sg9Fu)=YfV zNrcEkB3CP_%LRjDs~A3vQ=D%{t=Atd)wT>n$Pcu{JW2F;Ult6m7TCsJGKTT=oSr7Csnl zwW0y5llK|LQy|Yl#cd9KQ3w`I!LYzt1hXyWh~dpz!x-huMZGzuQn{$5>eO~SwM5Gh zF9icpB~*-t^0Oc*X22_^prX=vtgTD&HbHPy8?5T8M0CzLR}RF=5mU)K2$H!%l-61F zo;>f*7fIE@wq0abJ9|u};$2-V?d7WuE3P={s4XAY>kUCh4f;Y!*;CJlWirJD6Y$!m zHe(KbCgYVwQ`x5t6uaAzH)JUbw73*<8~n1vmMUH?aawad+g?5*6y=1cAY=&s0);2(z7MWTko$N9+)1lxus8z~QtW&KMA+NwzOGUhFaEYwW zB&u{KM!|M07idvlOSc?wa@xze8HP^9(JFa#8Okq8{+1!B&6!|}E~hC)^{5^vl}_Ta zQchM$uFTN7ls(mr2T}pO$=6X%16hv}Nw+c;vqeWiSO_?1UM+~NfVrEh`;$p2#ELo@ zhgHd*(WeWNb_`kZrJPw`suUCO5+BRqrE(fq{h})pu@nqyv8V>yalYYnW-DT=mSg$6 zVD=bfZO~>IL5=%ZeX&idl`8Ht!wa?{8;KOVVFwpA#Je)rsRo12Mx*W_P0qZhRL<75 zE^w)goQkE=IlK^7yj(r)CAvJt$yF&$neFkckh8g>T7yJu?kHW%a)jQIskG~4%aGT` zYuj50*;zV;TRi1*B3)?(1wT&5eYC4w=agJKrsf2{*&QXd6CS&su|@R-D{d|D26z!y zZMCSNh7wGUk7_NJZZ#d0y)3##yI`uZPCe<3@cy#wu7z`I%RtjEwq#{X^>P!ZB!d>7 zJBr!iahhpk9IukGOhl#X1<6+L7TtaWQ836nt@l=?fMo2LweDkuEay+S3u0RI*@M=w zw;G5>L(VW`?%E?Mqc_hL>ryCc*G@`oT~c3VliFLQKiKKGJvPoCr@Ee4#^}|4tTWwS8S#j24j*`wAMLVh@A!X79yQ3E1x>0>BRkVm%p%RNFMUfALJ$YleWS4ox zU88GlW7FLc@J!yXcV>JkCCV6#o`9OyW>C-;kGb_0W5_GR(M+Kka+Ct)5?6{rUf0IK za-JwC-d0v=w~MK44z?gXka+t6=34qAzcz zC_9}Ol$=`UWCM{ifKgM-ov0O~xfq?dMhb3H@Q?w2N%6Rn!8~QkXBzQbR(0{FT7?ey zyC#1|o6ELDT2#G;I_ruwOhleFL~5c)6he(66?9lS&6@U>Z}vvIaLLzG$!yL+b$xOy zVN4tR&6I(af-c1%3v?(>`HLR6F&+>Me5t1T=&l%NjqX%f>QX7r9#2KhX1U^MC{eeg z*&@hL#ACMesc@`N3aLuOEe115u0-1-dT-Rz2&av(X~@`ebhTT{X$S6|NUE7l*>fUi z54!AZnU_@95_IT9T?yFBWjvN=2s{=JChJr%?6!F7-h|KGHQ_SLIC2>;Sda>~V#nQ0 z2$53QZE%%q4qvtAmhv{m>$Qqxvd|FZT*4{mDgr61WW++tUOPwQay-+h=gaAGQY~d; z+PLDH8DpiXB&=@5XbMTGLd+a5H5p@5!WpKWaEqo$jQ7)d&Y!U+rFb)EAuPUR)Yu7@ z1?>RHfamL7GMhH0O}Z&netC3B5glI*{R4J!q3;N9>UUrdbqgZuW+#Y{bJN(PV+6v?H z;H`3#p%XdI+$e_l>`9PQP@8mtHIsxGh}GJKyaG{|P0{95E1S*eBMuL|{E;9RFDN{p z_eF5r=cdX6pXCW_((G%;?M&RqiB&lJD#McB>0|;Xc&)N~Ozwt4SS-9#ikO46fj7D( zvstu8gm}yCk43e^5M!g2PV2QJNQ=zWDulTxG#hTWb}p96bF?;c-7SN5ELF0~vZ`$_ z@vhwvfTPo}L0kUg$s}&9P=XRw!f~mkDU>^mrtc)pUF}@Wl{j+dYhm?_6x2%U)Vnds zNYc1b)yvwiad^llx&4lMOglbJRppYJi8L$$8?IT)wEUVi4`Wfa-LXli)H%FVkGBj& z(61>Z8@RE|XDu;+qaR;;n(ow2$bf6mu`OG=ZmUd*LdDc#97L8%b z+KL5jflxWzsc1!39JE$Z#*h05snC#f+R4AfheXj_EygL6wNnhFG(p-RYFT-nO{%t> z$?XZaY;3!u)hTR-SZ0M@)++DrQkj@LuazCO8cLdK(_zq@ciN0@o7w7w$V~~vq=<(+!X@lw&OjX?43>3BE zIJse{bSk5EU+-^cq$6j%Rjso_9U&U1qBG1Prsj=zSw}wugNMyPZR|?J*6Wy?< z*&byQ z9!Wy`dr65w`+I^d)ZdqE%RY4OKq4mRW9YPAB2q^O22xC=l|b3y4nvELs2{y@qMuVE ztrnV}nk4C1#_Pk;p@|CB2qfBQXg<-6(OR|P#Hh^@$sioxEuJv=;;5e<-}SD#^Skj0 zc(oYreeK{wNA8Y%<*|6oxmr{I_)~idj^&K#zw%%2??v8V!0?;>!^rlZ!Of2Tf!?z% z8%Oz(McU&z{R88Bk#qY85Vi}C*?t{77U8i3nq`33q=X|_uwSQXhmL56`v(@lL!^Ir zd~eURN5^Q=CmfO19m&yZK5E+nSTaxS*PjoPc>ip~sTJqyAJ#P8{li+ar+>C~@%GPt zvKQ$c9C&^v0+xo3Sen#7pydts&wdXsllvF;tiN^43dnx&h(kRO-8x1Of2^Pwn)s(t zJ0J(#S_HC@e%*rJ|M`2HUq4R3Ctj`D1oA(9Z&QH2n2y>i1mZIj$dWpNm~JPKtFr`h zcNfwu1hPLvAcr0%kabA{$%F`GC**N5fjr0&$o44cY$T8$+ytVP(7_3$TO^QiYe3&g zAm7;uWY2F2gokUPQtw3`TIy+9y0K^dB_{%Z+j0qB0c zgg|-%1hQErkmr{YND^$!T|glBp9lK45Xc)|0(rUwewcQGtss!=z{XxE>!{nn#|=>CdIEV3uCMyQKd8sZ z2O%BuZ^sDa7Lh=H3H7`Z?EM4U;W}uur=c#r&^{|6?fv@*6R!J+_=c{tCYD zpHCnI(5Ai{2*kh<$U^Y>w*>;3)6(jw>&uy@!5{xSrz3H;av?Ut|*$P*wt66(}g0)Bvv0R`#?wm*Zq27!Z+zZR~O z?jVqlz_)qV5ZZuq5Bh%^08n%oMTWG{J^kYU%ydE& z@1@7rE_rk>_UZHop6xyBMeP2a+uz2QzlJfNo_`53?rCh!2OqX)J+mA8>}>Xmy(hej zIRl>3iY;$q3;t4n?A!V8Vzc(gorU!uV|xrYum0%LH?RX+2Img#{0w6rpA#C3?ZW2W z8M*h`ZJ%LV8cKV`;Ahy58y3%dYt=q%(&o2E+%$b3cKNKABD3Uou?veI0Soi>GKWgP3jN2RU zzOeVtSo;3~DJn=eq$MUC=xe?p3`>#K=bxzM4nC++~ z(F2d|$E+v5blxe_I_&$^n@8QFY{Bg3?Y!Zyx%J|`_iVs67B=4dN@FiJ=c2D3o^kdrY|(dT zA6x&^2iU%)uTj(H@5FApra1nNtk2AEdBSTTPOYS4)(91 z^YYnGeCt234ZByF_M7V$9Kgn${LOJ<=UMF9W6u8jsaJoBUH9%US2w&guzyXkZ>lFecTZ*)HfG|-Pi%PPA#D7O$SXhm{WEO$p&OUZd2%<6 z-`~G`k@Gcd_S!+#cg%iFU3u0w5BNXCuD!`Py?Wy7*u<}QoO;P!2e4IFKM;KE{Liua z&^OE0yuKN`L>kB6|L{TV-h8cV`EQ$WAM{kKgKfc ztEQ&Te+A?BWSVE^_F#h}Ca?AP$hDd>H~|?PF>}Q52+r`Y8zVB**PuVGiS?f~pD=XSXe|$_^1p0KXT5v_X#S^(ZU59f(q@ncjrQL*PqWtF0UF4^ zYj6Ki|vto5rwbnL%H9Y2b`Ibmqm1T7ET`d762ZwuYfzNb;>YGm@% z;r^Ul7RA$AdAU)pRfoqJPP3k-pO=W>WWpb%km0z*%JpVmnOBi@{}j?0PP3n8n0G{R zo~+l+&Nj)tWc3)@;FUHkm9YHp^wrc9WSkbNQV86yzg#yCC8J G3;Y*Cc>^&3 literal 0 HcmV?d00001 diff --git a/sql/src/test/resources/drill/window/datasources/t_alltype.parquet.json b/sql/src/test/resources/drill/window/datasources/t_alltype.parquet.json new file mode 100644 index 00000000000..bef5694f7c7 --- /dev/null +++ b/sql/src/test/resources/drill/window/datasources/t_alltype.parquet.json @@ -0,0 +1,145 @@ +{"c1":1,"c2":592475043,"c3":616080519999272,"c4":"ObHeWTDEcbGzssDwPwurfs","c5":"0sZxIfZ CGwTOaLWZ6nWkUNx","c6":1456290852307,"c7":421426627200000,"c8":true,"c9":0.626179100469} +{"c1":2,"c2":1194343172,"c3":3136812789494,"c4":"bSrLdMAtGNmXsaFkmNpRaZnlPuUuSljiMqxJdgzoViIy","c5":"sluPVQz3991EyMTJL86YF8NEODzE1w2BtcKwPutL5TFUS7MM0c8pEYeWz2uq5pr0f0o0JcV5HRZs6gnwj3CPAdjisUk3LM5DvZcGUMkKpIEfziNrlcCrjRR10vWYVLeG6y5hG1ZsZgoSDWubDY2K1bZOaJTEP26AMDpYwIo","c6":1456290852308,"c7":420167260800000,"c8":true,"c9":0.0794280235964} +{"c1":3,"c2":-581053021,"c3":14567676734720,"c4":"zBmUwoxbiWhazyIFwkyTsCo","c5":"b4iHRz7x3jgWYQASSBusBaOS5KjIKpdxRqEOkVvZSqgwzwS3r1 5C81WnF5RWzLNDOi8qgJK0JrMMjVxOpFi13socmJ94NVelUSvb6umYkzBrZhpzYDFHZmXtAfWzRBkXfESrfYF2ME 5lI2v79oP54eucAvtIQApA1Bak8Fw9VgRznhGwE nKzuew1UpXk6dAW91ag099LyD29bViFQxWzCyf7FFDEy","c6":1456290852309,"c7":421521667200000,"c8":true,"c9":0.256111757801} +{"c1":4,"c2":719544129,"c3":26552329517384,"c4":"VlVwoLMKaardyJCdUttoBDIYVZ","c5":"TwDutiLBRsXHs0K0KvlzY6SHzNDJTpQFOigQ32YOYD7jCEpMJrT2SrvWBk9gJ7idoJhqIjW 4tPTI8u0cxFiEsU1IUNnONTYF5E3fGo8aXdA2Y0CAWqn4nHyY QPnVt3n5FnuQtQ88ayT3a0J86FOc641y4vCrTAqlN3jXVRJ1sg","c6":1456290852309,"c7":422532201600000,"c8":true,"c9":0.367266831393} +{"c1":5,"c2":-265354187,"c3":869394999808,"c4":"gwfrW","c5":"h79IKUK0fji8POwoapUs7YCKsC5WDOI1Oge25fe7QXMAQrxIgyEIhj3upa62rTJxOVA2vCMg6U 4koGrPZzda1aNjvGiK5Qjc0qUPnGCjoGdgjE87rfON4Uu02J5 Q7NiZ15CtSu8IuY6oqYyZclq47L6FC2grvhcuBTRiw9C1zgQ6d1523I2RCONHyYiZi7RrNhziA2tSZCPmI42GYHtjZ","c6":1456290852309,"c7":421190841600000,"c8":false,"c9":0.886914049583} +{"c1":6,"c2":-739409800,"c3":458112011400,"c4":"upOMZlypwcASQHtEkgpYybPwvjCtFHpuJkfNlQBSIrWqznQJEzEV","c5":"tmZET5agjU4yZaPXZDGW8iSPwRmkuGq4dpdkZqtv7edggkdZm3Im1j9okgoqqcuqPmo8r2EDK0bPZdk5YFXutZoAVhK5ecd06gGdNj2tWkSVKO7wqCUR2E7obM9ZYpZlgDecXTjzLvLUYuZy0TV sTjyt8DCU1QnKChPKD MbcopdpZkaJ5vWd5tIM8eYOB5kjdTiLfV3TVYb2LmZGt2f4xHbih odqbeq2ZDeEuFAwaoyL3UQI7Euj8","c6":1456290852310,"c7":421723670400000,"c8":true,"c9":0.492183947524} +{"c1":7,"c2":2022947631,"c3":122811962676,"c4":"ZAFOcferhjkcl","c5":"1wnKPdZ2a6nHOkC5kuo6lpCUtShnNjjjUd4f3vBs7AdhwQDoBPA CRMC7CCBTU VOx38gwZurb9HrSQNRrSd2kDuHB6WpqffMTwNfHPvqxnwHtbh4M8GwVEE0ocFsvYPkd4mIl7 uhdOOOS3H8jZpGa6Yt OoalDERUEcgZz","c6":1456290852311,"c7":420189379200000,"c8":false,"c9":0.60032636853} +{"c1":8,"c2":null,"c3":null,"c4":null,"c5":null,"c6":null,"c7":null,"c8":null,"c9":null} +{"c1":9,"c2":358750156,"c3":128037619138,"c4":"rNMsKIDnkuQFIYoJorACrHkFprLYMpp","c5":"fEwIVF31I9jk4C04qBYtw6pxswxIuThE3id0UuKTp6CgcnP9eNlVwJh82xvwsN 8CmX1RSQhtozud6Umv6kLCKSVF7BQbxRS07V78HgbrH0ZDV","c6":1456290852311,"c7":420977606400000,"c8":false,"c9":0.361570558696} +{"c1":10,"c2":-116481491,"c3":362751773072,"c4":"IcStZtjaSEfWDmrulvsnCwUQvfgn","c5":"aTISWde3pj5OJd9l7Zo7j9Kn8RRdWimrrPlc5BDFN3xpqrliH3Che9OJERMsQgwiafBwavLrTe21ORPrcml4Bh44fDbC4fBVvMMLTPVWbmgvmXm02Ct7A1WBMTM2C74LLJmIdZwPpwMXumO5dYZg8Z3KU2FnJ2bl80qqgyjJNlv9","c6":1456290852312,"c7":422991244800000,"c8":true,"c9":0.488525932471} +{"c1":11,"c2":993964305,"c3":713109032528,"c4":"TuFaJOrTexEQKCXg","c5":"9VLdcEruZzhtTbrz6Tu5vvfwwIy7jVtc7R5XEeZY","c6":1456291198301,"c7":420906067200000,"c8":true,"c9":0.957695228383} +{"c1":12,"c2":-1735090787,"c3":841324401984,"c4":"dPRmWDinhineYImsqZzRZSSMx","c5":"srBmkYpJY7TC0W","c6":1456291198302,"c7":420699571200000,"c8":false,"c9":0.707961195747} +{"c1":13,"c2":-101621245,"c3":4569922715352,"c4":"KkQGklqtAFOBwaNXDWkN","c5":"WMhbfG37BBZSwtryJVwMlS67TYe5UjqLYa5pHES13GC","c6":1456291198302,"c7":421197494400000,"c8":true,"c9":0.156305915074} +{"c1":14,"c2":202631122,"c3":6199873199432,"c4":"qV","c5":"w5Cs5Jas0T","c6":1456291198303,"c7":421541020800000,"c8":false,"c9":0.709658352873} +{"c1":15,"c2":-1800226884,"c3":2408646998880,"c4":"AtfZqknYOrYBXcFCORVnzcKmXyaEe","c5":"xfxBmCGvTcaBxEFXSvPQTaCZRCgE9Cus","c6":1456291198303,"c7":421215984000000,"c8":false,"c9":0.22155866245} +{"c1":16,"c2":2042714193,"c3":2437409008036,"c4":"BQyDpxxWldiVnQlFFfasgfwIIzpgJd","c5":"YF6qr4rAyBrHa94CosX6BWpfbwE8EXWXrupuFPoekm G1PE6vTKeliuW gxuq","c6":1456291198304,"c7":420776899200000,"c8":false,"c9":0.305618728365} +{"c1":17,"c2":-1236268475,"c3":90275297566,"c4":"OGnxbGFEaWMZhrJ","c5":"Tc0A7vQFGyzGXnoa90wXRVEMaZMmSTix2IqRw48WKqE50hCw","c6":1456291198305,"c7":423092073600000,"c8":true,"c9":0.141061129745} +{"c1":18,"c2":-1451268845,"c3":7456257725280,"c4":"HAUtIjkTOkHLzjhucQeLuH","c5":"a","c6":1456291198305,"c7":420537052800000,"c8":false,"c9":0.555072055925} +{"c1":19,"c2":-148845,"c3":3757280,"c4":"HALzjhucQeLuH","c5":null,"c6":1456291198305,"c7":419905900800000,"c8":false,"c9":0.455072055925} +{"c1":20,"c2":533611126,"c3":784427291198608,"c4":"JgPGWgjgNEjMXYYmWtC","c5":"PY250MG5pFd3NgQJGckqfC9mwY0DNazIAFmqRy1J","c6":1456291198305,"c7":420459120000000,"c8":false,"c9":0.355503179385} +{"c1":21,"c2":-642124822,"c3":43598566290864,"c4":"syewpw","c5":"iV24XUsay9jBU2wZgUoCErzIYo","c6":1456291198306,"c7":421299705600000,"c8":true,"c9":0.456846886729} +{"c1":22,"c2":735561305,"c3":6307081519934,"c4":"tMi","c5":"ig9ClFlYmzIAb99UFoGaUyjSorP9E9qA34km539T4so3qm9kRZZTZ","c6":1456291198306,"c7":422750188800000,"c8":true,"c9":0.740283135928} +{"c1":23,"c2":-350986527,"c3":69653384392040,"c4":"DkPvPKYuCdSatlPrzU","c5":"zfSgmg7tAawKVSJhkipg2nSNuprAp75eKDJry0focwPij J4hE","c6":1456291198311,"c7":420485040000000,"c8":false,"c9":0.336419256965} +{"c1":24,"c2":1847465545,"c3":38629268606556,"c4":"PV","c5":"D9lY2D4VJGmFffgSV9MoyojaAjNVxcVyBiEdHN911nwK EBRi2wwFuWwbnyLl","c6":1456291198312,"c7":420271459200000,"c8":false,"c9":0.964243969107} +{"c1":25,"c2":1442790109,"c3":61368071783056,"c4":"myUrrzUHGnxasz","c5":"wyweWUTyBPuo3jOBJLPfSWqji","c6":1456291198313,"c7":422857238400000,"c8":false,"c9":0.507695339931} +{"c1":26,"c2":-2125195491,"c3":57004017502044,"c4":"iqwqmwkUhrtTliZiPXRTstVpqgm","c5":"yCko0ouh0ueXmXK0O9 MPvLcCmlp7d7AI4ca1ru8d4jGGtbPc7mPx","c6":1456291198314,"c7":420243120000000,"c8":false,"c9":0.962960554859} +{"c1":27,"c2":-603256047,"c3":64691852740816,"c4":"lXEgAlS","c5":"7ceX9a22fDB1wb9RUb6TQWqvkbE6sXp2Sq0Lt","c6":1456291198314,"c7":420876604800000,"c8":false,"c9":0.940132365478} +{"c1":28,"c2":-1806928495,"c3":73151265354384,"c4":"yfjoc","c5":"sqxWGArGIX5jKOvvOI6BjG2l","c6":1456291198315,"c7":423006883200000,"c8":true,"c9":0.769634808418} +{"c1":29,"c2":-926147543,"c3":18430182,"c4":"kyTyhomxLi","c5":"9TR790gkKg3KMkc5k47V9QAuos2Ojig04QwTdO5nB","c6":1456291198315,"c7":421728768000000,"c8":false,"c9":0.754223352027} +{"c1":30,"c2":626932646,"c3":3285206109420,"c4":"IWZriOkCaYLDGOzMSp","c5":"oSv1RhKIOtLgOLe","c6":1456291198316,"c7":422313868800000,"c8":false,"c9":0.444467141108} +{"c1":31,"c2":-430498956,"c3":1258459028142,"c4":"V","c5":"Q1Am3KtwYAQx Vj7kk9nsawCmN BRciIQiN3hF","c6":1456291701220,"c7":420287529600000,"c8":true,"c9":0.210929121391} +{"c1":32,"c2":-938167176,"c3":6017016766800,"c4":"PStTin","c5":"dfOY 50EejHegPw9axCPz","c6":1456291701221,"c7":420197328000000,"c8":false,"c9":0.827238030237} +{"c1":33,"c2":-464056232,"c3":269431279554,"c4":"HHqSmnhL","c5":"VVRz QWFjn96AtBYO7eR001KZwKGLUrEqeMrPVcKSbQowuQ90XXFrl3iG","c6":1456291701221,"c7":422397676800000,"c8":true,"c9":0.764629411376} +{"c1":34,"c2":-1564810747,"c3":471032170712,"c4":"XJF","c5":"8D9e1bT5wwFpLD1Hr92VKaCQJS","c6":1456291701222,"c7":421392844800000,"c8":true,"c9":0.804630256964} +{"c1":35,"c2":2047004122,"c3":6190065899528,"c4":"PAobaowvKNSaPsKEdhLtCRWe","c5":"wGlZo1sVGPg","c6":1456291701222,"c7":422983382400000,"c8":false,"c9":0.119950105444} +{"c1":36,"c2":-682355573,"c3":879651728,"c4":"BANiwCtAOccuLIqNMVOQGBIQKNw","c5":"KxgxdOznNSNaltnFxkxj94qZ7C05fwQVUyBkN","c6":1456291701222,"c7":423138556800000,"c8":true,"c9":0.152121113454} +{"c1":37,"c2":1168899469,"c3":311335712,"c4":"bQoLkPBFAUXqcMpcp","c5":"6YE84Kb8G6Z8 HR9qkTp30a7cnBh3T rC1uWP4o3S7DSm","c6":1456291701222,"c7":422246476800000,"c8":false,"c9":0.674319849007} +{"c1":38,"c2":-1678398290,"c3":895040440,"c4":"CSavGyXMbAANckDiCshpuNzCmBXuHbHw","c5":"V VabLjrkDF4IJ7twnT9gqTv","c6":1456291701223,"c7":420476140800000,"c8":false,"c9":0.249230799416} +{"c1":39,"c2":-1433668371,"c3":102790112,"c4":"yytBlhCMRSmV","c5":"fanhgAi1Cla 3lwYXQTvWYdIvTF5ziIBc","c6":1456291701223,"c7":421929648000000,"c8":false,"c9":0.538207272457} +{"c1":40,"c2":1300934512,"c3":1526006148,"c4":"WYqpkjPLSeCA","c5":"4wK m9vhE XTHlpHv1U","c6":1456291701224,"c7":420625958400000,"c8":true,"c9":0.0125838026238} +{"c1":41,"c2":67513000,"c3":8652621344,"c4":"cTPjdg","c5":"KvYiWTtwEwEBTWUArtiUnPciZY V8VzRIVI03Gyt","c6":1456291701225,"c7":421659302400000,"c8":true,"c9":0.0616766496679} +{"c1":42,"c2":1071705345,"c3":774763920,"c4":"YfwqGtgVkBewvkIlQnEEnt","c5":"EtNWvXm45 Tw5CI9YjwflvI4zVqHUUpLal rAt2f8X0Wfp","c6":1456291701225,"c7":421992201600000,"c8":false,"c9":0.881191135812} +{"c1":43,"c2":-705176971,"c3":1305480928,"c4":"xLYbWdnPrlTIt","c5":"mGVBbGVhQkAhjQdAhiWAwHlCNN5f1wggwKxRjsBBbmlvodajek4E1r","c6":1456291701225,"c7":420234825600000,"c8":false,"c9":0.0963362944467} +{"c1":44,"c2":-790060808,"c3":2091106612,"c4":"chvsFysOhXbqRnniYOGeLUppugM","c5":"XZkwRGa0gxPwmnAG4VOQ1bNNgzM9cO1RzrcynKKBssqEP3BeEhibq","c6":1456291701226,"c7":420386025600000,"c8":false,"c9":0.292481581027} +{"c1":45,"c2":1221413457,"c3":8339791280,"c4":"SeQaEfXkDgumVjpOKOglGiXVe","c5":"CwZrVQVWdAfMtFcqWFJzjTcrnpWT48Rbkd7CO","c6":1456291701227,"c7":421768425600000,"c8":false,"c9":0.648820911939} +{"c1":46,"c2":627701102,"c3":33308856,"c4":"qaOEIqTfNGD","c5":"OoDOYgbcJ09XmxqPv1e","c6":1456291701227,"c7":422446579200000,"c8":true,"c9":0.40799240898} +{"c1":47,"c2":-841849915,"c3":42746339248,"c4":"YPiESbIjBms","c5":"xFfL8aYVA5EiiSWmvKkHAYRGYCiqctdxQwdkH4DOP","c6":1456291701228,"c7":420577660800000,"c8":false,"c9":0.324089746843} +{"c1":48,"c2":735186776,"c3":61478943200,"c4":"BJIPsOpsMTNRCO","c5":"80Z7O0f6A6fj1Beww7oOkwLSZdjO","c6":1456291701228,"c7":421328044800000,"c8":false,"c9":0.359520992037} +{"c1":49,"c2":56927672,"c3":84551700064,"c4":"Y","c5":"Wvu9T8ilDl8k3oaB0PFNjHirDoMdwv1YYCah6ahqiuATp1","c6":1456291701229,"c7":421813180800000,"c8":true,"c9":0.328379339497} +{"c1":50,"c2":1182933670,"c3":45692038432,"c4":"whFjvGloFQxwYmvLkshjwnADhUQlV","c5":"yt49EB4Y2RAnPQXNwTqfp7qzgDH1Jf1rjY0FnpIp2y4","c6":1456292105544,"c7":423012153600000,"c8":true,"c9":0.835195212647} +{"c1":51,"c2":980460718,"c3":5971386168,"c4":"OMJEMdywCTpQgqJUYDuKDxAPBkWdEP","c5":"akbcHSDflWlG8K6MFnIOOLBMZ4z8NV2FecgqpLO8","c6":1456292105544,"c7":420768691200000,"c8":true,"c9":0.0594494786115} +{"c1":52,"c2":-1263020365,"c3":17414114976,"c4":"vkUYqgXGGfELNjVwyYKVJMd","c5":"7zP0naU","c6":1456292105545,"c7":422403206400000,"c8":false,"c9":0.128559830476} +{"c1":53,"c2":-2078473628,"c3":23813625600,"c4":"PVAKCnVrihXStlfqJf","c5":"HKyyDE7cHRTzEQX2F","c6":1456292105545,"c7":421790371200000,"c8":true,"c9":0.497832439707} +{"c1":54,"c2":-388853860,"c3":4882213296,"c4":"NkSynIivNZiIplsAegulLrz","c5":"2AWYA664YJkqeGXGZeI8vf3fhX06wjhlUoRdzRBJnylTOnUsrz3LcyxbTyIQHr1YCMXl9jFx6LyhnaOYZorN4rOazR44zI2","c6":1456292105546,"c7":422153337600000,"c8":false,"c9":0.806218120908} +{"c1":55,"c2":147766593,"c3":3544613368,"c4":"ByZRmalvDsZD","c5":"XgulsUB2lwxhZaMz0POOTcxIAt7mPaBialPFYzth6Aa54OPBoTY9nlJpXWf89oIczO0QVg5lbkmQQUlljIMEe2Mh8rjtDoYox2uDykAXeH0BXGVtOZ12EIwe","c6":1456292105546,"c7":421184448000000,"c8":true,"c9":0.189172246469} +{"c1":56,"c2":420230528,"c3":7209802880,"c4":"ZwiFgCchxUqJTEShealG","c5":"K7RQD052LFrkPBfuL9hIihch","c6":1456292105546,"c7":422883417600000,"c8":true,"c9":0.462810537514} +{"c1":57,"c2":-1894216997,"c3":5384805744,"c4":"eBKZaQYxrdPMRRwgeqPtJjxjqOBqUH","c5":"z7cJERSftFvbtxxgVbdF31jZCHmkHBmazWaHAa3eKAp1PPEObxkcuhjA1vf6cSlGe4TT8Yektdwqcr1Jd","c6":1456292105546,"c7":420506121600000,"c8":true,"c9":0.604145500315} +{"c1":58,"c2":-263032861,"c3":56565941904,"c4":"dqXVKoNjMFKcIszKFRXCotmbejjxGU","c5":"3QDSogOdQYe6NfDPkEPIHhtECUKeT3eXsRdbwCj3omYIHz3HX","c6":1456292105547,"c7":420541459200000,"c8":false,"c9":0.472430725946} +{"c1":59,"c2":-1661674051,"c3":9595495792,"c4":"atHZ","c5":"7SWlwF1qaCNs3Jea1NollJfLsgTK95yuo","c6":1456292105547,"c7":421871846400000,"c8":true,"c9":0.0923867105652} +{"c1":60,"c2":-1129907976,"c3":6907332688,"c4":"Mc","c5":"4FN","c6":1456292105547,"c7":421541020800000,"c8":false,"c9":0.701638057307} +{"c1":61,"c2":-514254242,"c3":195633336,"c4":"kenCPo","c5":"eqVUbd9TzGrQS2W1HuV1apKCt1rkYLRTHgED2RIk1jhowGYzlRa2DVMpzTVc8RVO44rm9stOD9uvJPLNE48jVMQfhLwtNDlKXUchY8N0FOEsMW2G37X7dYnzod","c6":1456292105547,"c7":422432323200000,"c8":true,"c9":0.511369563541} +{"c1":62,"c2":null,"c3":null,"c4":null,"c5":null,"c6":null,"c7":null,"c8":null,"c9":null} +{"c1":63,"c2":-535361692,"c3":19818839912000624,"c4":"aAbGSv","c5":"yNVubV8dlc8KC8aijpwxEFyYSqtI8iiAQeZJAQMKUqTKMD5EqFSG4z3AraZ9dX","c6":1456292105548,"c7":422959968000000,"c8":true,"c9":0.587212121798} +{"c1":64,"c2":-960817944,"c3":295389276379024,"c4":"ToiqQVHAUTzNXZxhxT","c5":"pf7gyDzSkNhMD1Cio9GnoDpe7tFF6lPUfskrdSBrdmkPrwPN5YrF8cYArRXsqeiWiUwhbQ","c6":1456292105548,"c7":422200339200000,"c8":false,"c9":0.425544340933} +{"c1":65,"c2":2079413364,"c3":875274205296,"c4":"QiyvPpHnbIkkLsJeF","c5":"cuX2mutS6Dt23LpVfQ4RhuZ2upAZZxSDl3NQjHYecHX6UpRud5J5GUO2yKikEShe1AeqzPbW2W3GVn3pwpt5k9SFo5BbAQSgBpXWe7OUYuLB6XE0VPbSk6RK","c6":1456292105549,"c7":420572131200000,"c8":false,"c9":0.201374181269} +{"c1":66,"c2":-786819448,"c3":64868238648,"c4":"h","c5":"ScG74UmrI77csOCHrVQQMGnAtWw0IOpLtDTp6xF8eH2MxrvjwW5BhxTKxSelvvDK4LJlO0YhFW6CoZM62aUEkOmtXnTBp7KH4lAiCPW2lnC5DEHn","c6":1456292105549,"c7":420490656000000,"c8":true,"c9":0.795415967241} +{"c1":67,"c2":1250142164,"c3":6916992,"c4":"vpZxaEsZkTwnMmmWa","c5":"dQxtZCEAviuZYSdc4sgLjb","c6":1456292105549,"c7":422506281600000,"c8":false,"c9":0.556953353203} +{"c1":68,"c2":-38579995,"c3":233512622900,"c4":"JgwMVLyvDb","c5":"fwuYfa4PJlIQoxfetQdAaVY4VvbbX0N9E641","c6":1456292105550,"c7":423074275200000,"c8":false,"c9":0.453069633364} +{"c1":69,"c2":-1027828328,"c3":40672068432544096,"c4":"TZGIkEEfJb","c5":"RftjjeGNAAGDTikjbjJ6U4GxR8ZRGclneqPib9RyyXI37Rr8E3DstJ55jBpN4KY1j2Dp44l6NkRvDHQO","c6":1456292105550,"c7":422924976000000,"c8":false,"c9":0.998189105778} +{"c1":70,"c2":-569066973,"c3":8907959568528,"c4":"eah","c5":"2svWW0","c6":1456292148510,"c7":423076867200000,"c8":false,"c9":0.264360661376} +{"c1":71,"c2":815270244,"c3":15558122158,"c4":"l","c5":"5rZomJ4gkusDD","c6":1456292148510,"c7":420599520000000,"c8":true,"c9":0.85139020182} +{"c1":72,"c2":-2085514660,"c3":1345100636752,"c4":"Gb","c5":"VecY","c6":1456292148511,"c7":421304976000000,"c8":false,"c9":0.159955514577} +{"c1":73,"c2":-2086451715,"c3":778834561344,"c4":"LGGRD","c5":"Dv1","c6":1456292148511,"c7":422926185600000,"c8":false,"c9":0.892355433411} +{"c1":74,"c2":1317696149,"c3":13607285678,"c4":"Zf","c5":"h6rrw","c6":1456292148512,"c7":420486422400000,"c8":true,"c9":0.226484732} +{"c1":75,"c2":-475078365,"c3":591562831712,"c4":"Gas","c5":"Az81IOQGM1","c6":1456292148512,"c7":420418166400000,"c8":true,"c9":0.519357968945} +{"c1":76,"c2":-1610419012,"c3":987957004000,"c4":"u","c5":"RUUA2mTktOtMRpv","c6":1456292148512,"c7":421413148800000,"c8":true,"c9":0.283395173271} +{"c1":77,"c2":1,"c3":1,"c4":"gn","c5":"iyZ","c6":1456295748512,"c7":420525820800000,"c8":false,"c9":0.714975974474} +{"c1":78,"c2":841562550,"c3":5229353983640,"c4":"kfJpOk","c5":"BP4z","c6":1456292148512,"c7":420504998400000,"c8":true,"c9":0.773946465042} +{"c1":79,"c2":2049749227,"c3":7886072056192,"c4":"lJroaw","c5":"CS","c6":1456292148512,"c7":420713740800000,"c8":true,"c9":0.866341378246} +{"c1":80,"c2":-514969856,"c3":15673249928080,"c4":"BNnLZdPW","c5":"l5a0ZpK","c6":1456292148513,"c7":421434748800000,"c8":false,"c9":0.500069917482} +{"c1":81,"c2":-889738985,"c3":32298052657224,"c4":"xFDhe","c5":"yqzQp2PYA2w0r","c6":1456292148513,"c7":421726176000000,"c8":true,"c9":0.534138245169} +{"c1":82,"c2":743747735,"c3":42939563416968,"c4":"kcRx","c5":"ALGpDH9qQ","c6":1456292148513,"c7":420958080000000,"c8":true,"c9":0.0339015122383} +{"c1":83,"c2":-991380116,"c3":7521740669440,"c4":"I","c5":"VnrhwXNs5I7A","c6":1456292148513,"c7":420505689600000,"c8":false,"c9":0.729684203442} +{"c1":84,"c2":-605301438,"c3":6462434675584,"c4":"OXqgNP","c5":"XvH","c6":1456292148513,"c7":422746128000000,"c8":false,"c9":0.741169865031} +{"c1":85,"c2":-9600122,"c3":1150794800496,"c4":"XC","c5":"EZ5w","c6":1456292148545,"c7":421704144000000,"c8":false,"c9":0.00508533320078} +{"c1":86,"c2":-1304870155,"c3":7762439826812,"c4":"fOJJ","c5":"3kmqcAhqRv1JKH","c6":1456292148546,"c7":421705440000000,"c8":false,"c9":0.857763501506} +{"c1":87,"c2":-1151111063,"c3":3026615127030,"c4":"B","c5":"UsA2ijb5Q472","c6":1456292148546,"c7":422321299200000,"c8":true,"c9":0.852362483167} +{"c1":88,"c2":726080083,"c3":352777524,"c4":"pvjG","c5":"6DTPUc0kPXNDI","c6":1456292148546,"c7":422189884800000,"c8":true,"c9":0.499536294849} +{"c1":89,"c2":1269715757,"c3":298591968,"c4":"wUJle","c5":"P9hocYIzILG4d","c6":1456292148547,"c7":421664400000000,"c8":false,"c9":0.731465721495} +{"c1":90,"c2":469877353,"c3":5604128205368,"c4":"cgBBlTOr","c5":"0S09","c6":1456292148547,"c7":420358291200000,"c8":false,"c9":0.930318601251} +{"c1":91,"c2":1439825179,"c3":8448247315360,"c4":"GhHH","c5":"xU","c6":1456292148547,"c7":422529177600000,"c8":true,"c9":0.365706281943} +{"c1":92,"c2":-965564746,"c3":6461158733064,"c4":"ApKK","c5":"2MT6vQK7ILiMym","c6":1456292148548,"c7":422356982400000,"c8":true,"c9":0.160958193689} +{"c1":93,"c2":1396621364,"c3":5965143616184,"c4":"zavaYRc","c5":"m9Y","c6":1456292148548,"c7":421204924800000,"c8":true,"c9":0.123678441424} +{"c1":94,"c2":811012151,"c3":1275108040346,"c4":"yQmKx","c5":"ZHAGJpfsrd","c6":1456292148548,"c7":421316812800000,"c8":false,"c9":0.784623709976} +{"c1":95,"c2":1117245576,"c3":7044500976200,"c4":"fePHGR","c5":"pjLll4ngRT36ygIV","c6":1456292148548,"c7":421281993600000,"c8":false,"c9":0.187592141607} +{"c1":96,"c2":-450291430,"c3":7186198454512,"c4":"KhTYcGy","c5":"Y1l","c6":1456292148548,"c7":420275001600000,"c8":true,"c9":0.348668175704} +{"c1":97,"c2":0,"c3":2668027779964,"c4":"kdhbgC","c5":"ejaUw7d3DEIC7axy","c6":1456292148549,"c7":420673132800000,"c8":false,"c9":0.690933751147} +{"c1":98,"c2":-1807107154,"c3":667184558696,"c4":"C","c5":"M4GVt6c4s","c6":1456292148549,"c7":420653260800000,"c8":false,"c9":0.868842793576} +{"c1":99,"c2":-1610548040,"c3":269450722304,"c4":"UECld","c5":"DzTXQDe","c6":1456292148549,"c7":421828905600000,"c8":false,"c9":0.866249476344} +{"c1":100,"c2":1968262238,"c3":1853568964,"c4":"XY","c5":"O61h","c6":1456292148549,"c7":421527456000000,"c8":false,"c9":0.771913788526} +{"c1":101,"c2":547475512,"c3":5424751352,"c4":"Myf","c5":"u91","c6":1456292148549,"c7":421646774400000,"c8":false,"c9":0.020687569041} +{"c1":102,"c2":1347375329,"c3":7601674368,"c4":"PubTJ","c5":"B72D746BYbWS","c6":1456292148550,"c7":420430867200000,"c8":true,"c9":0.669628865357} +{"c1":103,"c2":-730200083,"c3":3734160392,"c4":"R","c5":"JF5b","c6":1456292148550,"c7":421339708800000,"c8":false,"c9":0.754789130107} +{"c1":104,"c2":-2145249349,"c3":8153964480,"c4":"BAGi","c5":"o69DyecndyD47","c6":1456292148550,"c7":421667078400000,"c8":true,"c9":0.669354481782} +{"c1":105,"c2":1922528937,"c3":36022570792,"c4":"lJy","c5":"8iirhhzlmMQeL","c6":1456292148550,"c7":422019849600000,"c8":false,"c9":0.53821290676} +{"c1":106,"c2":353908213,"c3":108831386830,"c4":"Z","c5":"vb32KaNk","c6":1456292148550,"c7":422921692800000,"c8":true,"c9":0.772742852356} +{"c1":107,"c2":-1,"c3":null,"c4":"kYEB","c5":"UY1wFk3nA6uIg","c6":1456292148550,"c7":423125078400000,"c8":false,"c9":0.00471118728446} +{"c1":108,"c2":465659139,"c3":196283469856,"c4":"RVVu","c5":"crs1mym","c6":1456292148550,"c7":422165001600000,"c8":true,"c9":0.510821715226} +{"c1":109,"c2":-1330396672,"c3":584831936,"c4":null,"c5":"XxXpO","c6":1456292302869,"c7":420166310400000,"c8":false,"c9":0.0730700338964} +{"c1":110,"c2":-220130197,"c3":17601020,"c4":"hjeac","c5":"n","c6":1456292302869,"c7":422323977600000,"c8":true,"c9":0.96826163121} +{"c1":111,"c2":-42902,"c3":562336,"c4":"f","c5":"LykOIn","c6":1456292302870,"c7":420630624000000,"c8":true,"c9":0.554463637243} +{"c1":112,"c2":765525961,"c3":28891422336265300,"c4":"demso","c5":"LZoFRza","c6":1456292302870,"c7":420894057600000,"c8":true,"c9":0.850373501413} +{"c1":113,"c2":-68207234,"c3":90427790806585824,"c4":"pnmrt","c5":"HFo","c6":null,"c7":420867878400000,"c8":true,"c9":0.71373237138} +{"c1":114,"c2":-1380251111,"c3":63163365737847936,"c4":"krg","c5":"JXzNCV","c6":1456292302871,"c7":421624569600000,"c8":true,"c9":0.445900639514} +{"c1":115,"c2":-1844719171,"c3":60499043191834664,"c4":"tdqfhi","c5":"SzMEux","c6":1456292302871,"c7":422194809600000,"c8":true,"c9":0.520455630248} +{"c1":116,"c2":1790522162,"c3":47372667161181496,"c4":"orbowc","c5":"n","c6":1456292302871,"c7":421364678400000,"c8":true,"c9":0.891198277906} +{"c1":117,"c2":-27319471,"c3":37102817894137256,"c4":"w","c5":"SYvkQ","c6":1456292302871,"c7":420918681600000,"c8":false,"c9":0.0359150523033} +{"c1":118,"c2":1317690445,"c3":61958708627376736,"c4":"aehmn","c5":"Vioq","c6":1456292302871,"c7":420640732800000,"c8":false,"c9":0.943878204382} +{"c1":119,"c2":1965293154,"c3":4216354032582922,"c4":"d","c5":"RSlQvgb","c6":1456292302872,"c7":420903561600000,"c8":true,"c9":0.300925886142} +{"c1":120,"c2":-1998776606,"c3":80843181252171408,"c4":"hntp","c5":"lt","c6":1456292302872,"c7":422202499200000,"c8":true,"c9":0.81436509096} +{"c1":121,"c2":-759066837,"c3":56811856992247296,"c4":"gbfr","c5":"RXwJTUQL","c6":1456292302872,"c7":null,"c8":true,"c9":0.602622967887} +{"c1":122,"c2":null,"c3":16079752775114834,"c4":"i","c5":"wOKgo","c6":1456292302872,"c7":421707340800000,"c8":false,"c9":0.314540418049} +{"c1":123,"c2":-1393583038,"c3":21198598572581100,"c4":"cg","c5":"jJo","c6":1456292302872,"c7":421884806400000,"c8":true,"c9":0.863833034127} +{"c1":124,"c2":-359360839,"c3":10812882337641318,"c4":"yn","c5":"pp","c6":1456292302872,"c7":422279222400000,"c8":true,"c9":0.984219215225} +{"c1":125,"c2":-135559442,"c3":26079995480527800,"c4":"u","c5":"PpZ","c6":1456292302872,"c7":421338672000000,"c8":true,"c9":0.112619968355} +{"c1":126,"c2":-963314220,"c3":39899141337524696,"c4":"r","c5":"Kv","c6":1456292302872,"c7":421093209600000,"c8":false,"c9":0.928182322637} +{"c1":127,"c2":-603033839,"c3":91979853046157776,"c4":"iknfwt","c5":"UxrG","c6":1456292302873,"c7":422923593600000,"c8":true,"c9":0.436065582677} +{"c1":128,"c2":1730933740,"c3":52988296769210224,"c4":null,"c5":"SQreQ","c6":1456292338864,"c7":420160435200000,"c8":null,"c9":0.709400581459} +{"c1":129,"c2":-1190592568,"c3":29537626363643852,"c4":null,"c5":"JNq","c6":1456292338864,"c7":421020892800000,"c8":false,"c9":0.426506472311} +{"c1":130,"c2":-1311152842,"c3":24893851443643300,"c4":null,"c5":"VZmvS","c6":1456292338865,"c7":421283203200000,"c8":true,"c9":0.891508292788} +{"c1":131,"c2":-612732312,"c3":14972217694439486,"c4":null,"c5":"yZYlsOH","c6":1456292338865,"c7":420777676800000,"c8":true,"c9":0.682577092573} +{"c1":132,"c2":-1948510563,"c3":2686369516214692,"c4":null,"c5":"HcwNvv","c6":1456292338865,"c7":420190416000000,"c8":true,"c9":0.496484780895} +{"c1":133,"c2":1726823975,"c3":64270113732818952,"c4":null,"c5":"UPYQXFGA","c6":1456292338865,"c7":421703625600000,"c8":true,"c9":0.858728636519} +{"c1":134,"c2":220535371,"c3":15937600070688400,"c4":null,"c5":"tTsISx","c6":1456292338866,"c7":421422134400000,"c8":null,"c9":0.0978174122941} +{"c1":135,"c2":-278647794,"c3":52598911986023288,"c4":null,"c5":"JzQY","c6":1456292338866,"c7":421918416000000,"c8":false,"c9":0.0918752610245} +{"c1":136,"c2":288414270,"c3":8695637273720187,"c4":null,"c5":"OpFxaJUK","c6":1456292338866,"c7":420629328000000,"c8":true,"c9":0.0241314164497} +{"c1":137,"c2":-1033256154,"c3":2397384354833930,"c4":null,"c5":null,"c6":1456292369484,"c7":421578086400000,"c8":true,"c9":0.817785665204} +{"c1":138,"c2":-461761396,"c3":78042258136748336,"c4":null,"c5":null,"c6":1456292369484,"c7":421621632000000,"c8":true,"c9":0.514550507111} +{"c1":139,"c2":-1863053037,"c3":35173898947893320,"c4":null,"c5":null,"c6":1456292369485,"c7":420712531200000,"c8":true,"c9":0.641830766022} +{"c1":140,"c2":-775202711,"c3":21011901540311080,"c4":null,"c5":null,"c6":1456292369485,"c7":420698448000000,"c8":false,"c9":0.563296650508} +{"c1":141,"c2":-109815917,"c3":46096729108954488,"c4":null,"c5":null,"c6":1456292369485,"c7":421707772800000,"c8":null,"c9":0.82789510574} +{"c1":142,"c2":1733733471,"c3":84276421148786976,"c4":null,"c5":null,"c6":1456292369485,"c7":420706828800000,"c8":true,"c9":0.737905542986} +{"c1":143,"c2":1559768019,"c3":91381489058732016,"c4":null,"c5":null,"c6":1456292369486,"c7":422574019200000,"c8":true,"c9":null} +{"c1":144,"c2":-1054306811,"c3":null,"c4":null,"c5":null,"c6":1456292369486,"c7":421684876800000,"c8":false,"c9":0.273335960277} +{"c1":145,"c2":1580853588,"c3":17990322900862228,"c4":null,"c5":null,"c6":1456292369486,"c7":422222025600000,"c8":false,"c9":0.668501595682} \ No newline at end of file diff --git a/sql/src/test/resources/drill/window/datasources/tblWnulls.parquet b/sql/src/test/resources/drill/window/datasources/tblWnulls.parquet new file mode 100644 index 0000000000000000000000000000000000000000..1bed27d9645c90c1179e0e6d7300d31b0e8d6594 GIT binary patch literal 395 zcmZXRze)o^5XNWLowzwK#%N}}g~LKfI4}Vd6fCw-3$YB=HfIvFk!VcnJb)lp_Y}TE zTFX3zPa&RjSEyj|oBi1R=Fjd9&aQe4{DJ(A9rhwCY~o1gdA=xPZVdoDU9U@|<#HR> zEQ(Y@P0+Y|Crd?W2!*#Q1VT{)joFYqy%oyFPVEFXYev_B5}iZU94-J5PS5LbdjKH4 zGsW58F>oAVS`P4xM`uw#&{t+&16(0 zl_6#2{H2~wCNtK+s-+?|>;Z?R=(hl@cd_fVO)So1Qvp{PSXpo1lzmZc LogicalScan -> LogicalProject(NTILE) -> LogicalScan -> TableReference \ No newline at end of file diff --git a/sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_2.e b/sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_2.e new file mode 100644 index 00000000000..ecaa73c0403 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_2.e @@ -0,0 +1,22 @@ +5000 127.0 63.99 1 +9223372036854775807 63.99 256.0 1 +65534 256.0 255.9993 1 +-1 255.9993 -128.0 1 +1 -128.0 -33.0 2 +17 -33.0 29.13 2 +1000 29.13 1.0 2 +200 1.0 -1.0 2 +4611686018427387903 -1.0 128.978 3 +1001 128.978 123.129 3 +30 123.129 null 3 +-65535 13.9 -256.0 1 +10000000 -256.0 128.9 1 +0 128.9 32.9 1 +13 32.9 13.79 1 +23 13.79 100.09 2 +9999999 100.09 0.0 2 +197 0.0 99.9 2 +9223372036854775806 99.9 69.89 2 +92233720385475807 69.89 255.99 3 +25 255.99 127.9 3 +3000 127.9 null 3 diff --git a/sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_2.q b/sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_2.q new file mode 100644 index 00000000000..43083a8a068 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/aggregates/aggOWnFn_2.q @@ -0,0 +1 @@ +SELECT col1 ,col2 ,LEAD(col2) OVER windw LEAD_col2, tile FROM (SELECT col0 , col1 , col2, col7 , NTILE(3) over windw tile from "allTypsUniq.parquet" WINDOW windw as (PARTITION BY col7 ORDER BY col0)) sub_query WINDOW windw as (PARTITION BY col7 ORDER BY col0) \ No newline at end of file From d02bb8bb6e1e6d0aae3ebdb06f2f1ea335b8ad6d Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Thu, 6 Jul 2023 11:13:32 -0700 Subject: [PATCH 72/74] Set explain attributes after the query is prepared (#14490) * Add support for DML WITH AS. * One more UT for with as subquery. * Add a test with join query * Use root query prepared node instead of individual SqlNode types. - Set the explain plan attributes after the query is prepared when the query is planned and we've the finalized output names in the root source rel node. - Adjust tests; add unit test for negative ordinal case. - Remove the exception / error handling logic from resolveClusteredBy function since the validations now happen before it comes to the function * Update comment. --- .../calcite/parser/DruidSqlParserUtils.java | 86 +++----- .../sql/calcite/planner/DruidPlanner.java | 3 - .../sql/calcite/planner/IngestHandler.java | 4 +- .../sql/calcite/planner/QueryHandler.java | 1 + .../sql/calcite/CalciteInsertDmlTest.java | 203 ++++++++++++++++++ .../sql/calcite/CalciteReplaceDmlTest.java | 38 ++++ .../parser/DruidSqlParserUtilsTest.java | 176 +++++---------- 7 files changed, 334 insertions(+), 177 deletions(-) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java index 36fe62e2db3..0cd11b5e020 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java @@ -32,13 +32,11 @@ import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlNumericLiteral; -import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlOrderBy; -import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.SqlTimestampLiteral; -import org.apache.calcite.sql.SqlUtil; import org.apache.calcite.sql.dialect.CalciteSqlDialect; import org.apache.calcite.tools.ValidationException; +import org.apache.calcite.util.Pair; import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.StringUtils; @@ -327,67 +325,41 @@ public class DruidSqlParserUtils * * *

    - * The function will return the following clusteredBy columns for the above SQL: ["__time", "page_alias", "FLOOR(\"cost\")", cityName"] + * The function will return the following clusteredBy columns for the above SQL: ["__time", "page_alias", "FLOOR(\"cost\")", cityName"]. * Any ordinal and expression specified in the CLUSTERED BY clause will resolve to the final output column name. *

    + *

    + * This function must be called after the query is prepared when all the validations are complete, including {@link #validateClusteredByColumns}, + * so we can safely access the arguments. + *

    * @param clusteredByNodes List of {@link SqlNode}s representing columns to be clustered by. - * @param sourceNode The select or order by source node. + * @param sourceFieldMappings The source field output mappings extracted from the validated root query rel node post prepare phase. + * */ @Nullable - public static List resolveClusteredByColumnsToOutputColumns(SqlNodeList clusteredByNodes, SqlNode sourceNode) + public static List resolveClusteredByColumnsToOutputColumns( + final SqlNodeList clusteredByNodes, + final ImmutableList> sourceFieldMappings + ) { // CLUSTERED BY is an optional clause if (clusteredByNodes == null) { return null; } - Preconditions.checkArgument( - sourceNode instanceof SqlSelect || sourceNode instanceof SqlOrderBy, - "Source node must be either SqlSelect or SqlOrderBy, but found [%s]", - sourceNode == null ? null : sourceNode.getKind() - ); - - final SqlSelect selectNode = (sourceNode instanceof SqlSelect) ? (SqlSelect) sourceNode - : (SqlSelect) ((SqlOrderBy) sourceNode).query; - final List selectList = selectNode.getSelectList().getList(); final List retClusteredByNames = new ArrayList<>(); for (SqlNode clusteredByNode : clusteredByNodes) { - if (SqlUtil.isLiteral(clusteredByNode)) { - // The node is a literal number -- an ordinal is specified in the CLUSTERED BY clause. Validate and lookup the - // ordinal in the select list. - int ordinal = ((SqlNumericLiteral) clusteredByNode).getValueAs(Integer.class); - if (ordinal < 1 || ordinal > selectList.size()) { - throw InvalidSqlInput.exception( - "Ordinal[%d] specified in the CLUSTERED BY clause is invalid. It must be between 1 and %d.", - ordinal, - selectList.size() - ); - } - SqlNode node = selectList.get(ordinal - 1); - - if (node instanceof SqlBasicCall) { - retClusteredByNames.add(getColumnNameFromSqlCall(node)); - } else { - Preconditions.checkArgument( - node instanceof SqlIdentifier, - "Node must be a SqlIdentifier, but found [%s]", - node.getKind() - ); - SqlIdentifier n = ((SqlIdentifier) node); - retClusteredByNames.add(n.isSimple() ? n.getSimple() : n.names.get(1)); - } + if (clusteredByNode instanceof SqlNumericLiteral) { + // The node is a literal number -- an ordinal in the CLUSTERED BY clause. Lookup the ordinal in field mappings. + final int ordinal = ((SqlNumericLiteral) clusteredByNode).getValueAs(Integer.class); + retClusteredByNames.add(sourceFieldMappings.get(ordinal - 1).right); } else if (clusteredByNode instanceof SqlBasicCall) { // The node is an expression/operator. - retClusteredByNames.add(getColumnNameFromSqlCall(clusteredByNode)); + retClusteredByNames.add(getColumnNameFromSqlCall((SqlBasicCall) clusteredByNode)); } else { - // The node is a simple SqlIdentifier, add the name. - Preconditions.checkArgument( - clusteredByNode instanceof SqlIdentifier, - "ClusteredBy node must be a SqlIdentifier, but found [%s]", - clusteredByNode.getKind() - ); + // For everything else, just return the simple string representation of the node. retClusteredByNames.add(clusteredByNode.toString()); } } @@ -395,16 +367,12 @@ public class DruidSqlParserUtils return retClusteredByNames; } - private static String getColumnNameFromSqlCall(final SqlNode sqlCallNode) + private static String getColumnNameFromSqlCall(final SqlBasicCall sqlCallNode) { - Preconditions.checkArgument(sqlCallNode instanceof SqlBasicCall, "Node must be a SqlBasicCall type"); - // The node may be an alias or expression, in which case we'll get the output name - SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlCallNode; - SqlOperator operator = (sqlBasicCall).getOperator(); - if (operator instanceof SqlAsOperator) { + if (sqlCallNode.getOperator() instanceof SqlAsOperator) { // Get the output name for the alias operator. - SqlNode sqlNode = (sqlBasicCall).getOperandList().get(1); + SqlNode sqlNode = sqlCallNode.getOperandList().get(1); return sqlNode.toString(); } else { // Return the expression as-is. @@ -430,6 +398,18 @@ public class DruidSqlParserUtils clusteredByNode ); } + + // Calcite already throws Ordinal out of range exception for positive non-existent ordinals. This negative ordinal check + // is for completeness and is fixed in later Calcite versions. + if (clusteredByNode instanceof SqlNumericLiteral) { + final int ordinal = ((SqlNumericLiteral) clusteredByNode).getValueAs(Integer.class); + if (ordinal < 1) { + throw InvalidSqlInput.exception( + "Ordinal [%d] specified in the CLUSTERED BY clause is invalid. It must be a positive integer.", + ordinal + ); + } + } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java index 7d7bdc8d545..490f220ea9e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java @@ -151,9 +151,6 @@ public class DruidPlanner implements Closeable handler = createHandler(root); handler.validate(); plannerContext.setResourceActions(handler.resourceActions()); - if (root.getKind() == SqlKind.EXPLAIN) { - plannerContext.setExplainAttributes(handler.explainAttributes()); - } state = State.VALIDATED; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java index 2ce19acbbac..3d38c6b3f2c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java @@ -277,7 +277,7 @@ public abstract class IngestHandler extends QueryHandler DruidSqlInsert.OPERATOR.getName(), targetDatasource, ingestionGranularity, - DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(sqlNode.getClusteredBy(), sqlNode.getSource()), + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(sqlNode.getClusteredBy(), rootQueryRel.fields), null ); } @@ -351,7 +351,7 @@ public abstract class IngestHandler extends QueryHandler DruidSqlReplace.OPERATOR.getName(), targetDatasource, ingestionGranularity, - DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(sqlNode.getClusteredBy(), sqlNode.getSource()), + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(sqlNode.getClusteredBy(), rootQueryRel.fields), replaceIntervals ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index ac3c6faff18..bb313222dc9 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -162,6 +162,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand final RelDataType returnedRowType; if (explain != null) { + handlerContext.plannerContext().setExplainAttributes(explainAttributes()); returnedRowType = getExplainStructType(typeFactory); } else { returnedRowType = returnedRowType(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index 71672ae8d51..395543f0820 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -755,6 +755,209 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest didTest = true; } + @Test + public void testExplainPlanInsertWithAsSubQueryClusteredBy() + { + skipVectorize(); + + final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"foo\",\"partitionedBy\":{\"type\":\"all\"},\"clusteredBy\":[\"namespace\",\"country\"]}"; + + final String sql = "EXPLAIN PLAN FOR\n" + + "INSERT INTO \"foo\"\n" + + "WITH dd AS (\n" + + "SELECT * FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"inline\",\"data\":\"{\\\" \\\": 1681794225551, \\\"namespace\\\": \\\"day1\\\", \\\"country\\\": \\\"one\\\"}\\n{\\\"__time\\\": 1681794225558, \\\"namespace\\\": \\\"day2\\\", \\\"country\\\": \\\"two\\\"}\"}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"name\":\"__time\",\"type\":\"long\"},{\"name\":\"namespace\",\"type\":\"string\"},{\"name\":\"country\",\"type\":\"string\"}]'\n" + + " )\n" + + "))\n" + + "\n" + + "SELECT\n" + + " __time,\n" + + " namespace,\n" + + " country\n" + + "FROM dd\n" + + "PARTITIONED BY ALL\n" + + "CLUSTERED BY 2, 3"; + + final String legacyExplanation = "DruidQueryRel(" + + "query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\"," + + "\"inputSource\":{\"type\":\"inline\",\"data\":\"{\\\" \\\": 1681794225551, \\\"namespace\\\": \\\"day1\\\", \\\"country\\\": \\\"one\\\"}\\n" + + "{\\\"__time\\\": 1681794225558, \\\"namespace\\\": \\\"day2\\\", \\\"country\\\": \\\"two\\\"}\"}," + + "\"inputFormat\":{\"type\":\"json\",\"keepNullColumns\":false,\"assumeNewlineDelimited\":false,\"useJsonNodeReader\":false}," + + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]}," + + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + + "\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}]," + + "\"columns\":[\"__time\",\"country\",\"namespace\"],\"legacy\":false,\"context\":{\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," + + "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}]," + + " signature=[{__time:LONG, namespace:STRING, country:STRING}])\n"; + + // Use testQuery for EXPLAIN (not testIngestionQuery). + testQuery( + PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, + ImmutableMap.of("sqlQueryId", "dummy"), + Collections.emptyList(), + sql, + CalciteTests.SUPER_USER_AUTH_RESULT, + ImmutableList.of(), + new DefaultResultsVerifier( + ImmutableList.of( + new Object[]{ + legacyExplanation, + resources, + attributes + } + ), + null + ), + null + ); + + // Test correctness of the query when only the CLUSTERED BY clause is present + final String explanation = "[{\"query\":{\"queryType\":\"scan\"," + "\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\"," + + "\"data\":\"{\\\" \\\": 1681794225551, \\\"namespace\\\": \\\"day1\\\", \\\"country\\\": \\\"one\\\"}\\n" + + "{\\\"__time\\\": 1681794225558, \\\"namespace\\\": \\\"day2\\\", \\\"country\\\": \\\"two\\\"}\"}," + + "\"inputFormat\":{\"type\":\"json\",\"keepNullColumns\":false,\"assumeNewlineDelimited\":false,\"useJsonNodeReader\":false}," + + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]}," + + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + + "\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}]," + + "\"columns\":[\"__time\",\"country\",\"namespace\"],\"legacy\":false,\"context\":{\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," + + "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," + + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]," + + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"namespace\",\"outputColumn\":\"namespace\"}," + + "{\"queryColumn\":\"country\",\"outputColumn\":\"country\"}]}]"; + + testQuery( + PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, + ImmutableMap.of("sqlQueryId", "dummy"), + Collections.emptyList(), + sql, + CalciteTests.SUPER_USER_AUTH_RESULT, + ImmutableList.of(), + new DefaultResultsVerifier( + ImmutableList.of( + new Object[]{ + explanation, + resources, + attributes + } + ), + null + ), + null + ); + + // Not using testIngestionQuery, so must set didTest manually to satisfy the check in tearDown. + didTest = true; + } + + @Test + public void testExplainPlanInsertJoinQuery() + { + skipVectorize(); + + final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"my_table\",\"type\":\"DATASOURCE\"}]"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"my_table\",\"partitionedBy\":\"HOUR\",\"clusteredBy\":[\"__time\",\"isRobotAlias\",\"countryCapital\",\"regionName\"]}"; + + final String sql = "EXPLAIN PLAN FOR\n" + + "INSERT INTO my_table\n" + + "WITH\n" + + "wikidata AS (SELECT * FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"name\":\"isRobot\",\"type\":\"string\"},{\"name\":\"timestamp\",\"type\":\"string\"},{\"name\":\"cityName\",\"type\":\"string\"},{\"name\":\"countryIsoCode\",\"type\":\"string\"},{\"name\":\"regionName\",\"type\":\"string\"}]'\n" + + " )\n" + + ")),\n" + + "countries AS (SELECT * FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]}',\n" + + " '{\"type\":\"tsv\",\"findColumnsFromHeader\":true}',\n" + + " '[{\"name\":\"Country\",\"type\":\"string\"},{\"name\":\"Capital\",\"type\":\"string\"}," + + "{\"name\":\"ISO3\",\"type\":\"string\"},{\"name\":\"ISO2\",\"type\":\"string\"}]'\n" + + " )\n" + + "))\n" + + "SELECT\n" + + " TIME_PARSE(\"timestamp\") AS __time,\n" + + " isRobot AS isRobotAlias,\n" + + " countries.Capital AS countryCapital,\n" + + " regionName\n" + + "FROM wikidata\n" + + "LEFT JOIN countries ON wikidata.countryIsoCode = countries.ISO2\n" + + "PARTITIONED BY HOUR\n" + + "CLUSTERED BY 1, 2, 3, regionName"; + + final String legacyExplanation = "DruidJoinQueryRel(condition=[=($3, $6)], joinType=[left], query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"}," + + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\"," + + "\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"}," + + "{\"columnName\":\"Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"Capital\",\"isRobot\",\"regionName\",\"v0\"],\"legacy\":false,\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\"," + + "\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{v0:LONG, isRobot:STRING, Capital:STRING, regionName:STRING}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\",\"keepNullColumns\":false,\"assumeNewlineDelimited\":false," + + "\"useJsonNodeReader\":false},\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}]}," + + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"cityName\",\"countryIsoCode\",\"isRobot\",\"regionName\",\"timestamp\"],\"legacy\":false," + + "\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{isRobot:STRING, timestamp:STRING, cityName:STRING, countryIsoCode:STRING, regionName:STRING}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true}," + + "\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + + "\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"legacy\":false,\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{Capital:STRING, ISO2:STRING}])\n"; + // Use testQuery for EXPLAIN (not testIngestionQuery). + testQuery( + PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, + ImmutableMap.of("sqlQueryId", "dummy"), + Collections.emptyList(), + sql, + CalciteTests.SUPER_USER_AUTH_RESULT, + ImmutableList.of(), + new DefaultResultsVerifier( + ImmutableList.of( + new Object[]{ + legacyExplanation, + resources, + attributes + } + ), + null + ), + null + ); + + // Test correctness of the query when only the CLUSTERED BY clause is present + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\",\"keepNullColumns\":false," + + "\"assumeNewlineDelimited\":false,\"useJsonNodeReader\":false},\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"}," + + "{\"name\":\"regionName\",\"type\":\"STRING\"}]},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true}," + + "\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\"," + + "\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"legacy\":false,\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"," + + "\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}},\"rightPrefix\":\"j0.\",\"condition\":\"(\\\"countryIsoCode\\\" == \\\"j0.ISO2\\\")\",\"joinType\":\"LEFT\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + + "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"}," + + "{\"columnName\":\"j0.Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"isRobot\",\"j0.Capital\",\"regionName\",\"v0\"],\"legacy\":false,\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"," + + "\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"j0.Capital\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"__time\"}," + + "{\"queryColumn\":\"isRobot\",\"outputColumn\":\"isRobotAlias\"},{\"queryColumn\":\"j0.Capital\",\"outputColumn\":\"countryCapital\"},{\"queryColumn\":\"regionName\",\"outputColumn\":\"regionName\"}]}]"; + + testQuery( + PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, + ImmutableMap.of("sqlQueryId", "dummy"), + Collections.emptyList(), + sql, + CalciteTests.SUPER_USER_AUTH_RESULT, + ImmutableList.of(), + new DefaultResultsVerifier( + ImmutableList.of( + new Object[]{ + explanation, + resources, + attributes + } + ), + null + ), + null + ); + + // Not using testIngestionQuery, so must set didTest manually to satisfy the check in tearDown. + didTest = true; + } + @Test public void testExplainPlanInsertWithClusteredByDescThrowsException() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index 282ea7d8251..c4f9503d4c7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -936,6 +936,44 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest .verify(); } + @Test + public void testReplaceWithNonExistentOrdinalInClusteredBy() + { + skipVectorize(); + + final String sql = "REPLACE INTO dst" + + " OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00' " + + " SELECT * FROM foo" + + " PARTITIONED BY DAY" + + " CLUSTERED BY 1, 2, 100"; + + testIngestionQuery() + .sql(sql) + .expectValidationError( + invalidSqlContains("Ordinal out of range") + ) + .verify(); + } + + @Test + public void testReplaceWithNegativeOrdinalInClusteredBy() + { + skipVectorize(); + + final String sql = "REPLACE INTO dst" + + " OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00' " + + " SELECT * FROM foo" + + " PARTITIONED BY DAY" + + " CLUSTERED BY 1, -2, 3 DESC"; + + testIngestionQuery() + .sql(sql) + .expectValidationError( + invalidSqlIs("Ordinal [-2] specified in the CLUSTERED BY clause is invalid. It must be a positive integer.") + ) + .verify(); + } + @Test public void testReplaceFromExternalProjectSort() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java index b47e5bfeaa3..4abc210b830 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtilsTest.java @@ -29,11 +29,10 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; -import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlPostfixOperator; -import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.util.Pair; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.granularity.Granularities; @@ -144,49 +143,47 @@ public class DruidSqlParserUtilsTest @Test public void testNullClusteredBy() { - final SqlNodeList selectArgs = new SqlNodeList(SqlParserPos.ZERO); - selectArgs.add(new SqlIdentifier("__time", new SqlParserPos(0, 1))); - Assert.assertNull(DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns( - null, - new SqlSelect(SqlParserPos.ZERO, null, selectArgs, null, null, null, null, null, null, null, null) - ) + final ImmutableList> fields = ImmutableList.of( + Pair.of(1, "__time"), + Pair.of(2, "foo"), + Pair.of(3, "bar") + ); + Assert.assertNull( + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns( + null, + fields + ) ); } @Test - public void testNullSource() + public void testSimpledClusteredByWithNullSource() { final SqlNodeList args = new SqlNodeList(SqlParserPos.ZERO); args.add(new SqlIdentifier("__time", SqlParserPos.ZERO)); - args.add(new SqlIntervalQualifier(TimeUnit.DAY, null, SqlParserPos.ZERO)); - - IllegalArgumentException iae = Assert.assertThrows( - IllegalArgumentException.class, - () -> DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(args, null) + args.add(new SqlIdentifier("FOO", new SqlParserPos(0, 2))); + SqlBasicCall sqlBasicCall1 = new SqlBasicCall( + new SqlAsOperator(), + new SqlNode[]{ + new SqlIdentifier("DIM3", SqlParserPos.ZERO), + new SqlIdentifier("DIM3_ALIAS", SqlParserPos.ZERO) + }, + new SqlParserPos(0, 3) + ); + args.add(sqlBasicCall1); + Assert.assertEquals( + Arrays.asList("__time", "FOO", "DIM3_ALIAS"), + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(args, null) ); - Assert.assertEquals("Source node must be either SqlSelect or SqlOrderBy, but found [null]", iae.getMessage()); } @Test public void testSimpleClusteredBy() { - final SqlNodeList selectArgs = new SqlNodeList(SqlParserPos.ZERO); - selectArgs.add(new SqlIdentifier("__time", new SqlParserPos(0, 1))); - selectArgs.add(new SqlIdentifier("FOO", new SqlParserPos(0, 2))); - selectArgs.add(new SqlIdentifier("BOO", new SqlParserPos(0, 3))); - - final SqlSelect sqlSelect = new SqlSelect( - SqlParserPos.ZERO, - null, - selectArgs, - null, - null, - null, - null, - null, - null, - null, - null + final ImmutableList> sourceFieldMappings = ImmutableList.of( + Pair.of(1, "__time"), + Pair.of(2, "FOO"), + Pair.of(3, "BOO") ); final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); @@ -196,42 +193,7 @@ public class DruidSqlParserUtilsTest Assert.assertEquals( Arrays.asList("__time", "FOO", "BOO"), - DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, sqlSelect) - ); - } - - @Test - public void testClusteredByOrdinalInvalidThrowsException() - { - final SqlNodeList selectArgs = new SqlNodeList(SqlParserPos.ZERO); - selectArgs.add(new SqlIdentifier("__time", new SqlParserPos(0, 1))); - selectArgs.add(new SqlIdentifier("FOO", new SqlParserPos(0, 2))); - selectArgs.add(new SqlIdentifier("BOO", new SqlParserPos(0, 3))); - - final SqlSelect sqlSelect = new SqlSelect( - SqlParserPos.ZERO, - null, - selectArgs, - null, - null, - null, - null, - null, - null, - null, - null - ); - - final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); - clusteredByArgs.add(new SqlIdentifier("__time", SqlParserPos.ZERO)); - clusteredByArgs.add(new SqlIdentifier("FOO", SqlParserPos.ZERO)); - clusteredByArgs.add(SqlLiteral.createExactNumeric("4", SqlParserPos.ZERO)); - - MatcherAssert.assertThat( - Assert.assertThrows(DruidException.class, () -> DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, sqlSelect)), - DruidExceptionMatcher.invalidSqlInput().expectMessageIs( - "Ordinal[4] specified in the CLUSTERED BY clause is invalid. It must be between 1 and 3." - ) + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, sourceFieldMappings) ); } @@ -272,18 +234,14 @@ public class DruidSqlParserUtilsTest args3.add(SqlLiteral.createCharString("PT1H", SqlParserPos.ZERO)); selectArgs.add(TimeFloorOperatorConversion.SQL_FUNCTION.createCall(args3)); - final SqlSelect sqlSelect = new SqlSelect( - SqlParserPos.ZERO, - null, - selectArgs, - null, - null, - null, - null, - null, - null, - null, - null + final ImmutableList> sourceFieldMappings = ImmutableList.of( + Pair.of(1, "__time"), + Pair.of(2, "DIM3"), + Pair.of(3, "DIM3_ALIAS"), + Pair.of(4, "floor_dim4_time"), + Pair.of(5, "DIM5"), + Pair.of(5, "DIM6"), + Pair.of(7, "TIME_FLOOR(\"timestamps\", 'PT1H')") ); // Construct the clustered by args @@ -295,45 +253,7 @@ public class DruidSqlParserUtilsTest Assert.assertEquals( Arrays.asList("DIM3_ALIAS", "floor_dim4_time", "DIM5", "TIME_FLOOR(\"timestamps\", 'PT1H')"), - DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, sqlSelect) - ); - } - - @Test - public void testSimpleClusteredByWithOrderBy() - { - final SqlNodeList selectArgs = new SqlNodeList(SqlParserPos.ZERO); - selectArgs.add(new SqlIdentifier("__time", new SqlParserPos(0, 1))); - selectArgs.add(new SqlIdentifier("FOO", new SqlParserPos(0, 2))); - selectArgs.add(new SqlIdentifier("BOO", new SqlParserPos(0, 3))); - - final SqlSelect sqlSelect = new SqlSelect( - SqlParserPos.ZERO, - null, - selectArgs, - null, - null, - null, - null, - null, - null, - null, - null - ); - - SqlNodeList orderList = new SqlNodeList(SqlParserPos.ZERO); - orderList.add(sqlSelect); - - SqlNode orderByNode = new SqlOrderBy(SqlParserPos.ZERO, sqlSelect, orderList, null, null); - - final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); - clusteredByArgs.add(new SqlIdentifier("__time", SqlParserPos.ZERO)); - clusteredByArgs.add(new SqlIdentifier("FOO", SqlParserPos.ZERO)); - clusteredByArgs.add(SqlLiteral.createExactNumeric("3", SqlParserPos.ZERO)); - - Assert.assertEquals( - Arrays.asList("__time", "FOO", "BOO"), - DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, orderByNode) + DruidSqlParserUtils.resolveClusteredByColumnsToOutputColumns(clusteredByArgs, sourceFieldMappings) ); } } @@ -390,6 +310,24 @@ public class DruidSqlParserUtilsTest .expectMessageIs("Invalid CLUSTERED BY clause [`DIM4` DESC]: cannot sort in descending order.") .assertThrowsAndMatches(() -> DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs)); } + + /** + * Tests clause "CLUSTERED BY DIM1, DIM2, 3, -10" + */ + @Test + public void testClusteredByColumnsWithNegativeOrdinalThrowsException() + { + final SqlNodeList clusteredByArgs = new SqlNodeList(SqlParserPos.ZERO); + clusteredByArgs.add(new SqlIdentifier("DIM1", SqlParserPos.ZERO)); + clusteredByArgs.add(new SqlIdentifier("DIM2", SqlParserPos.ZERO)); + clusteredByArgs.add(new SqlIdentifier("3", SqlParserPos.ZERO)); + clusteredByArgs.add(SqlLiteral.createExactNumeric("-10", SqlParserPos.ZERO)); + + DruidExceptionMatcher + .invalidSqlInput() + .expectMessageIs("Ordinal [-10] specified in the CLUSTERED BY clause is invalid. It must be a positive integer.") + .assertThrowsAndMatches(() -> DruidSqlParserUtils.validateClusteredByColumns(clusteredByArgs)); + } } public static class FloorToGranularityConversionErrorsTest From 037f09bef2224b0912b0f225c7653929dd98af88 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 6 Jul 2023 11:51:04 -0700 Subject: [PATCH 73/74] HttpRemoteTaskRunner: Fix markLazyWorkers for maxLazyWorkers == 0. (#14532) --- .../druid/indexing/overlord/RemoteTaskRunner.java | 6 +++--- .../druid/indexing/overlord/WorkerTaskRunner.java | 5 ++++- .../indexing/overlord/hrtr/HttpRemoteTaskRunner.java | 11 +++++++++-- .../overlord/hrtr/HttpRemoteTaskRunnerTest.java | 11 +++++++++++ 4 files changed, 27 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java index c3c00e2f0e6..10e66395996 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/RemoteTaskRunner.java @@ -1397,11 +1397,11 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer } @Override - public Collection markWorkersLazy(Predicate isLazyWorker, int maxWorkers) + public Collection markWorkersLazy(Predicate isLazyWorker, int maxLazyWorkers) { // skip the lock and bail early if we should not mark any workers lazy (e.g. number // of current workers is at or below the minNumWorkers of autoscaler config) - if (maxWorkers < 1) { + if (maxLazyWorkers < 1) { return Collections.emptyList(); } // status lock is used to prevent any tasks being assigned to the worker while we mark it lazy @@ -1412,7 +1412,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer if (getAssignedTasks(zkWorker.getWorker()).isEmpty() && isLazyWorker.apply(zkWorker.toImmutable())) { log.info("Adding Worker[%s] to lazySet!", zkWorker.getWorker().getHost()); lazyWorkers.put(worker.getKey(), zkWorker); - if (lazyWorkers.size() == maxWorkers) { + if (lazyWorkers.size() == maxLazyWorkers) { // only mark excess workers as lazy and allow their cleanup break; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunner.java index 6520de2111f..9ef9065dc8f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunner.java @@ -48,8 +48,11 @@ public interface WorkerTaskRunner extends TaskRunner /** * Check which workers can be marked as lazy + * + * @param isLazyWorker predicate that checks if a worker is lazy + * @param maxLazyWorkers maximum number of lazy workers to return */ - Collection markWorkersLazy(Predicate isLazyWorker, int maxWorkers); + Collection markWorkersLazy(Predicate isLazyWorker, int maxLazyWorkers); WorkerTaskRunnerConfig getConfig(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java index a6354c90628..7d0d210736f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java @@ -96,6 +96,7 @@ import java.io.InputStream; import java.net.URL; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -922,8 +923,14 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer } @Override - public Collection markWorkersLazy(Predicate isLazyWorker, int maxWorkers) + public Collection markWorkersLazy(Predicate isLazyWorker, int maxLazyWorkers) { + // skip the lock and bail early if we should not mark any workers lazy (e.g. number + // of current workers is at or below the minNumWorkers of autoscaler config) + if (maxLazyWorkers < 1) { + return Collections.emptyList(); + } + synchronized (statusLock) { for (Map.Entry worker : workers.entrySet()) { final WorkerHolder workerHolder = worker.getValue(); @@ -931,7 +938,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer if (isWorkerOkForMarkingLazy(workerHolder.getWorker()) && isLazyWorker.apply(workerHolder.toImmutable())) { log.info("Adding Worker[%s] to lazySet!", workerHolder.getWorker().getHost()); lazyWorkers.put(worker.getKey(), workerHolder); - if (lazyWorkers.size() == maxWorkers) { + if (lazyWorkers.size() == maxLazyWorkers) { // only mark excess workers as lazy and allow their cleanup break; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java index 6cc6329c584..d7feece444c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java @@ -1105,6 +1105,17 @@ public class HttpRemoteTaskRunnerTest Assert.assertEquals(task1.getId(), Iterables.getOnlyElement(taskRunner.getRunningTasks()).getTaskId()); Assert.assertEquals(task2.getId(), Iterables.getOnlyElement(taskRunner.getPendingTasks()).getTaskId()); + Assert.assertEquals( + Collections.emptyList(), + taskRunner.markWorkersLazy(Predicates.alwaysTrue(), 0) + ); + + Assert.assertEquals( + "host3:8080", + Iterables.getOnlyElement(taskRunner.markWorkersLazy(Predicates.alwaysTrue(), 1)) + .getHost() + ); + Assert.assertEquals( "host3:8080", Iterables.getOnlyElement(taskRunner.markWorkersLazy(Predicates.alwaysTrue(), Integer.MAX_VALUE)) From dd78e00dc543609db2c75c23f9546fc46cb86ad8 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 6 Jul 2023 15:10:59 -0700 Subject: [PATCH 74/74] Fix ColumnSignature error message and jdk17 test issue. (#14538) * Fix ColumnSignature error message and jdk17 test issue. On jdk17, the "problem" part of the error message could change from NullPointerException to: Cannot invoke "String.length()" because "s" is null Due to the new more-helpful NPEs in Java 17. This broke the expectation and led to test failures on this case. This patch fixes the problem by improving the error message so it isn't a generic NullPointerException. * Fix format. --- .../java/org/apache/druid/segment/column/ColumnSignature.java | 2 +- .../java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnSignature.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnSignature.java index 62553ddaf4e..2ea9f960606 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnSignature.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnSignature.java @@ -50,7 +50,7 @@ class ColumnSignature // Name must be nonnull, but type can be null (if the type is unknown) if (name == null || name.isEmpty()) { - throw new IAE(name, "Column name must be non-empty"); + throw new IAE("Column name must be provided and non-empty"); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index 395543f0820..1a19f9ddf70 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -1638,7 +1638,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest "general" ) .expectMessageContains( - "Cannot construct instance of `org.apache.druid.segment.column.ColumnSignature`, problem: `java.lang.NullPointerException`\n" + "Cannot construct instance of `org.apache.druid.segment.column.ColumnSignature`, problem: Column name must be provided and non-empty" ) ) .verify();