You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by "Rahul Kumar (JIRA)" <ji...@apache.org> on 2017/05/16 07:34:04 UTC

[jira] [Commented] (CARBONDATA-1054) Data Load failure in sort_column table.

    [ https://issues.apache.org/jira/browse/CARBONDATA-1054?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16011896#comment-16011896 ] 

Rahul Kumar commented on CARBONDATA-1054:
-----------------------------------------

[~vin7149] can you please try above query with following properties: 

enable.offheap.sort=true
enable.unsafe.sort=true
carbon.load.use.batch.sort=true

> Data Load failure in sort_column table.
> ---------------------------------------
>
>                 Key: CARBONDATA-1054
>                 URL: https://issues.apache.org/jira/browse/CARBONDATA-1054
>             Project: CarbonData
>          Issue Type: Bug
>          Components: data-load
>         Environment: Spark 2.1
>            Reporter: Vinod Rohilla
>            Priority: Trivial
>         Attachments: newdata.csv, Thriftserver_Log_Data_Load
>
>
> Error displays to user while load the Data in table.
> Step to Reproduce:
> 1: Create table:
> CREATE TABLE sorttable4_offheap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, empname');
> 2:Load Data in table:
> LOAD DATA local inpath 'hdfs://localhost:54310/newdata.csv' INTO TABLE sorttable4_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"');
> 3:Error on Beeline:
> 0: jdbc:hive2://localhost:10000> LOAD DATA local inpath 'hdfs://localhost:54310/newdata.csv' INTO TABLE sorttable4_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"');
> Error: java.lang.Exception: DataLoad failure: Problem while shutdown the server (state=,code=0)
> 4: ThriftServer logs:
> 17/05/15 18:07:07 INFO thriftserver.SparkExecuteStatementOperation: Running query 'LOAD DATA local inpath 'hdfs://localhost:54310/newdata.csv' INTO TABLE sorttable4_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')' with f01a8190-d2c8-40e1-b932-af695cf0129c
> 17/05/15 18:07:07 INFO parser.CarbonSparkSqlParser: Parsing command: LOAD DATA local inpath 'hdfs://localhost:54310/newdata.csv' INTO TABLE sorttable4_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')
> 17/05/15 18:07:07 INFO optimizer.CarbonLateDecodeRule: pool-24-thread-10 Skip CarbonOptimizer
> 17/05/15 18:07:07 INFO locks.HdfsFileLock: pool-24-thread-10 HDFS lock path:hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/meta.lock
> 17/05/15 18:07:07 INFO command.LoadTable: pool-24-thread-10 Successfully able to get the table metadata file lock
> 17/05/15 18:07:07 INFO command.LoadTable: pool-24-thread-10 Initiating Direct Load for the Table : (vinod.sorttable4_offheap_inmemory)
> 17/05/15 18:07:07 INFO util.GlobalDictionaryUtil$: pool-24-thread-10 Generate global dictionary from source data files!
> 17/05/15 18:07:07 INFO memory.MemoryStore: Block broadcast_5 stored as values in memory (estimated size 238.9 KB, free 2.4 GB)
> 17/05/15 18:07:07 INFO memory.MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 23.4 KB, free 2.4 GB)
> 17/05/15 18:07:07 INFO storage.BlockManagerInfo: Added broadcast_5_piece0 in memory on 192.168.2.179:46491 (size: 23.4 KB, free: 2.5 GB)
> 17/05/15 18:07:07 INFO spark.SparkContext: Created broadcast 5 from NewHadoopRDD at GlobalDictionaryUtil.scala:381
> 17/05/15 18:07:07 INFO optimizer.CarbonLateDecodeRule: pool-24-thread-10 Skip CarbonOptimizer
> 17/05/15 18:07:07 INFO spark.SparkContext: Starting job: collect at GlobalDictionaryUtil.scala:746
> 17/05/15 18:07:07 INFO input.FileInputFormat: Total input paths to process : 1
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: Registering RDD 26 (RDD at CarbonGlobalDictionaryRDD.scala:274)
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: Got job 2 (collect at GlobalDictionaryUtil.scala:746) with 4 output partitions
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: Final stage: ResultStage 4 (collect at GlobalDictionaryUtil.scala:746)
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: Parents of final stage: List(ShuffleMapStage 3)
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: Missing parents: List(ShuffleMapStage 3)
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: Submitting ShuffleMapStage 3 (CarbonBlockDistinctValuesCombineRDD[26] at RDD at CarbonGlobalDictionaryRDD.scala:274), which has no missing parents
> 17/05/15 18:07:07 INFO memory.MemoryStore: Block broadcast_6 stored as values in memory (estimated size 24.1 KB, free 2.4 GB)
> 17/05/15 18:07:07 INFO memory.MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 10.0 KB, free 2.4 GB)
> 17/05/15 18:07:07 INFO storage.BlockManagerInfo: Added broadcast_6_piece0 in memory on 192.168.2.179:46491 (size: 10.0 KB, free: 2.5 GB)
> 17/05/15 18:07:07 INFO spark.SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:996
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 3 (CarbonBlockDistinctValuesCombineRDD[26] at RDD at CarbonGlobalDictionaryRDD.scala:274)
> 17/05/15 18:07:07 INFO scheduler.TaskSchedulerImpl: Adding task set 3.0 with 1 tasks
> 17/05/15 18:07:07 INFO scheduler.TaskSetManager: Starting task 0.0 in stage 3.0 (TID 6, localhost, executor driver, partition 0, ANY, 6384 bytes)
> 17/05/15 18:07:07 INFO executor.Executor: Running task 0.0 in stage 3.0 (TID 6)
> 17/05/15 18:07:07 INFO rdd.NewHadoopRDD: Input split: hdfs://localhost:54310/newdata.csv:0+1663250
> 17/05/15 18:07:07 INFO executor.Executor: Finished task 0.0 in stage 3.0 (TID 6). 1643 bytes result sent to driver
> 17/05/15 18:07:07 INFO scheduler.TaskSetManager: Finished task 0.0 in stage 3.0 (TID 6) in 136 ms on localhost (executor driver) (1/1)
> 17/05/15 18:07:07 INFO scheduler.TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool 
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: ShuffleMapStage 3 (RDD at CarbonGlobalDictionaryRDD.scala:274) finished in 0.136 s
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: looking for newly runnable stages
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: running: Set()
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: waiting: Set(ResultStage 4)
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: failed: Set()
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: Submitting ResultStage 4 (CarbonGlobalDictionaryGenerateRDD[28] at RDD at CarbonGlobalDictionaryRDD.scala:335), which has no missing parents
> 17/05/15 18:07:07 INFO memory.MemoryStore: Block broadcast_7 stored as values in memory (estimated size 6.8 KB, free 2.4 GB)
> 17/05/15 18:07:07 INFO memory.MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 3.7 KB, free 2.4 GB)
> 17/05/15 18:07:07 INFO storage.BlockManagerInfo: Added broadcast_7_piece0 in memory on 192.168.2.179:46491 (size: 3.7 KB, free: 2.5 GB)
> 17/05/15 18:07:07 INFO spark.SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:996
> 17/05/15 18:07:07 INFO scheduler.DAGScheduler: Submitting 4 missing tasks from ResultStage 4 (CarbonGlobalDictionaryGenerateRDD[28] at RDD at CarbonGlobalDictionaryRDD.scala:335)
> 17/05/15 18:07:07 INFO scheduler.TaskSchedulerImpl: Adding task set 4.0 with 4 tasks
> 17/05/15 18:07:07 INFO scheduler.TaskSetManager: Starting task 0.0 in stage 4.0 (TID 7, localhost, executor driver, partition 0, ANY, 6121 bytes)
> 17/05/15 18:07:07 INFO scheduler.TaskSetManager: Starting task 1.0 in stage 4.0 (TID 8, localhost, executor driver, partition 1, ANY, 6121 bytes)
> 17/05/15 18:07:07 INFO scheduler.TaskSetManager: Starting task 2.0 in stage 4.0 (TID 9, localhost, executor driver, partition 2, ANY, 6121 bytes)
> 17/05/15 18:07:07 INFO scheduler.TaskSetManager: Starting task 3.0 in stage 4.0 (TID 10, localhost, executor driver, partition 3, ANY, 6121 bytes)
> 17/05/15 18:07:07 INFO executor.Executor: Running task 0.0 in stage 4.0 (TID 7)
> 17/05/15 18:07:07 INFO executor.Executor: Running task 1.0 in stage 4.0 (TID 8)
> 17/05/15 18:07:07 INFO executor.Executor: Running task 2.0 in stage 4.0 (TID 9)
> 17/05/15 18:07:07 INFO locks.HdfsFileLock: Executor task launch worker-5 HDFS lock path:hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/e8c4c615-25f3-4f9f-92e8-760a8f36118b.lock
> 17/05/15 18:07:07 INFO storage.ShuffleBlockFetcherIterator: Getting 1 non-empty blocks out of 1 blocks
> 17/05/15 18:07:07 INFO storage.ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms
> 17/05/15 18:07:07 INFO locks.HdfsFileLock: Executor task launch worker-4 HDFS lock path:hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/da3e9354-8951-4e1b-bb46-57a4cfd59c2e.lock
> 17/05/15 18:07:07 INFO executor.Executor: Running task 3.0 in stage 4.0 (TID 10)
> 17/05/15 18:07:07 INFO storage.ShuffleBlockFetcherIterator: Getting 1 non-empty blocks out of 1 blocks
> 17/05/15 18:07:07 INFO storage.ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms
> 17/05/15 18:07:07 INFO locks.HdfsFileLock: Executor task launch worker-7 HDFS lock path:hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/1afdd308-aaf4-4e7d-8eee-b12c0f2ef76b.lock
> 17/05/15 18:07:07 INFO storage.ShuffleBlockFetcherIterator: Getting 1 non-empty blocks out of 1 blocks
> 17/05/15 18:07:07 INFO storage.ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms
> 17/05/15 18:07:07 INFO locks.HdfsFileLock: Executor task launch worker-6 HDFS lock path:hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/2f15d866-ece2-4639-a656-14beb6172c3a.lock
> 17/05/15 18:07:07 INFO storage.ShuffleBlockFetcherIterator: Getting 1 non-empty blocks out of 1 blocks
> 17/05/15 18:07:07 INFO storage.ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Successfully able to get the dictionary lock for designation
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Executor task launch worker-5 
>  columnName: designation
>  columnId: e8c4c615-25f3-4f9f-92e8-760a8f36118b
>  new distinct values count: 0
>  combine lists: 2
>  create dictionary cache: 2
>  sort list, distinct and write: 3
>  write sort info: 0
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Successfully able to get the dictionary lock for empname
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Executor task launch worker-4 
>  columnName: empname
>  columnId: da3e9354-8951-4e1b-bb46-57a4cfd59c2e
>  new distinct values count: 0
>  combine lists: 4
>  create dictionary cache: 2
>  sort list, distinct and write: 1
>  write sort info: 0
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Successfully able to get the dictionary lock for deptname
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Successfully able to get the dictionary lock for workgroupcategoryname
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Executor task launch worker-6 
>  columnName: workgroupcategoryname
>  columnId: 2f15d866-ece2-4639-a656-14beb6172c3a
>  new distinct values count: 0
>  combine lists: 1
>  create dictionary cache: 1
>  sort list, distinct and write: 1
>  write sort info: 0
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Executor task launch worker-7 
>  columnName: deptname
>  columnId: 1afdd308-aaf4-4e7d-8eee-b12c0f2ef76b
>  new distinct values count: 0
>  combine lists: 2
>  create dictionary cache: 4
>  sort list, distinct and write: 2
>  write sort info: 0
> 17/05/15 18:07:08 INFO locks.HdfsFileLock: Executor task launch worker-5 Deleted the lock file hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/e8c4c615-25f3-4f9f-92e8-760a8f36118b.lock
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Dictionary designation Unlocked Successfully.
> 17/05/15 18:07:08 INFO executor.Executor: Finished task 1.0 in stage 4.0 (TID 8). 1678 bytes result sent to driver
> 17/05/15 18:07:08 INFO scheduler.TaskSetManager: Finished task 1.0 in stage 4.0 (TID 8) in 110 ms on localhost (executor driver) (1/4)
> 17/05/15 18:07:08 INFO locks.HdfsFileLock: Executor task launch worker-4 Deleted the lock file hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/da3e9354-8951-4e1b-bb46-57a4cfd59c2e.lock
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Dictionary empname Unlocked Successfully.
> 17/05/15 18:07:08 INFO executor.Executor: Finished task 0.0 in stage 4.0 (TID 7). 1678 bytes result sent to driver
> 17/05/15 18:07:08 INFO scheduler.TaskSetManager: Finished task 0.0 in stage 4.0 (TID 7) in 122 ms on localhost (executor driver) (2/4)
> 17/05/15 18:07:08 INFO locks.HdfsFileLock: Executor task launch worker-6 Deleted the lock file hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/2f15d866-ece2-4639-a656-14beb6172c3a.lock
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Dictionary workgroupcategoryname Unlocked Successfully.
> 17/05/15 18:07:08 INFO locks.HdfsFileLock: Executor task launch worker-7 Deleted the lock file hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/1afdd308-aaf4-4e7d-8eee-b12c0f2ef76b.lock
> 17/05/15 18:07:08 INFO rdd.CarbonGlobalDictionaryGenerateRDD: Dictionary deptname Unlocked Successfully.
> 17/05/15 18:07:08 INFO executor.Executor: Finished task 2.0 in stage 4.0 (TID 9). 1678 bytes result sent to driver
> 17/05/15 18:07:08 INFO executor.Executor: Finished task 3.0 in stage 4.0 (TID 10). 1678 bytes result sent to driver
> 17/05/15 18:07:08 INFO scheduler.TaskSetManager: Finished task 2.0 in stage 4.0 (TID 9) in 131 ms on localhost (executor driver) (3/4)
> 17/05/15 18:07:08 INFO scheduler.TaskSetManager: Finished task 3.0 in stage 4.0 (TID 10) in 130 ms on localhost (executor driver) (4/4)
> 17/05/15 18:07:08 INFO scheduler.TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool 
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: ResultStage 4 (collect at GlobalDictionaryUtil.scala:746) finished in 0.128 s
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: Job 2 finished: collect at GlobalDictionaryUtil.scala:746, took 0.289833 s
> 17/05/15 18:07:08 INFO util.GlobalDictionaryUtil$: pool-24-thread-10 generate global dictionary successfully
> 17/05/15 18:07:08 AUDIT rdd.CarbonDataRDDFactory$: [vinod-Vostro-3559][anonymous][Thread-179]Data load request has been received for table vinod.sorttable4_offheap_inmemory
> 17/05/15 18:07:08 INFO util.CommonUtil$: pool-24-thread-10 [Block Distribution]
> 17/05/15 18:07:08 INFO util.CommonUtil$: pool-24-thread-10 totalInputSpaceConsumed: 1663250 , defaultParallelism: 4
> 17/05/15 18:07:08 INFO util.CommonUtil$: pool-24-thread-10 mapreduce.input.fileinputformat.split.maxsize: 16777216
> 17/05/15 18:07:08 INFO input.FileInputFormat: Total input paths to process : 1
> 17/05/15 18:07:08 INFO hive.DistributionUtil$: pool-24-thread-10 Executors configured : 1
> 17/05/15 18:07:08 INFO hive.DistributionUtil$: pool-24-thread-10 Total Time taken to ensure the required executors : 1
> 17/05/15 18:07:08 INFO hive.DistributionUtil$: pool-24-thread-10 Time elapsed to allocate the required executors: 0
> 17/05/15 18:07:08 INFO rdd.CarbonDataRDDFactory$: pool-24-thread-10 Total Time taken in block allocation: 1
> 17/05/15 18:07:08 INFO rdd.CarbonDataRDDFactory$: pool-24-thread-10 Total no of blocks: 1, No.of Nodes: 1
> 17/05/15 18:07:08 INFO rdd.CarbonDataRDDFactory$: pool-24-thread-10 #Node: vinod-Vostro-3559 no.of.blocks: 1
> 17/05/15 18:07:08 INFO memory.MemoryStore: Block broadcast_8 stored as values in memory (estimated size 58.0 MB, free 2.4 GB)
> 17/05/15 18:07:08 INFO memory.MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 23.3 KB, free 2.4 GB)
> 17/05/15 18:07:08 INFO storage.BlockManagerInfo: Added broadcast_8_piece0 in memory on 192.168.2.179:46491 (size: 23.3 KB, free: 2.5 GB)
> 17/05/15 18:07:08 INFO spark.SparkContext: Created broadcast 8 from broadcast at NewCarbonDataLoadRDD.scala:185
> 17/05/15 18:07:08 INFO spark.SparkContext: Starting job: collect at CarbonDataRDDFactory.scala:630
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: Got job 3 (collect at CarbonDataRDDFactory.scala:630) with 1 output partitions
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: Final stage: ResultStage 5 (collect at CarbonDataRDDFactory.scala:630)
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: Parents of final stage: List()
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: Missing parents: List()
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: Submitting ResultStage 5 (NewCarbonDataLoadRDD[29] at RDD at NewCarbonDataLoadRDD.scala:174), which has no missing parents
> 17/05/15 18:07:08 INFO rdd.NewCarbonDataLoadRDD: Preferred Location for split : vinod-Vostro-3559
> 17/05/15 18:07:08 INFO memory.MemoryStore: Block broadcast_9 stored as values in memory (estimated size 12.0 KB, free 2.4 GB)
> 17/05/15 18:07:08 INFO memory.MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 6.1 KB, free 2.4 GB)
> 17/05/15 18:07:08 INFO storage.BlockManagerInfo: Added broadcast_9_piece0 in memory on 192.168.2.179:46491 (size: 6.1 KB, free: 2.5 GB)
> 17/05/15 18:07:08 INFO spark.SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:996
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (NewCarbonDataLoadRDD[29] at RDD at NewCarbonDataLoadRDD.scala:174)
> 17/05/15 18:07:08 INFO scheduler.TaskSchedulerImpl: Adding task set 5.0 with 1 tasks
> 17/05/15 18:07:08 INFO scheduler.TaskSetManager: Starting task 0.0 in stage 5.0 (TID 11, localhost, executor driver, partition 0, ANY, 6650 bytes)
> 17/05/15 18:07:08 INFO executor.Executor: Running task 0.0 in stage 5.0 (TID 11)
> 17/05/15 18:07:08 INFO rdd.NewCarbonDataLoadRDD: Input split: vinod-Vostro-3559
> 17/05/15 18:07:08 INFO rdd.NewCarbonDataLoadRDD: The Block Count in this node :1
> 17/05/15 18:07:08 INFO newflow.AbstractDataLoadProcessorStep: Thread-105 Rows processed in step Input Processor : 0
> 17/05/15 18:07:08 INFO newflow.AbstractDataLoadProcessorStep: Thread-107 Rows processed in step Sort Processor : 0
> 17/05/15 18:07:08 INFO newflow.AbstractDataLoadProcessorStep: Thread-108 Rows processed in step Data Writer : 0
> 17/05/15 18:07:08 INFO newflow.AbstractDataLoadProcessorStep: Thread-106 Rows processed in step Data Converter : 0
> 17/05/15 18:07:08 INFO sortdata.SortParameters: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] Sort size for table: 500000
> 17/05/15 18:07:08 INFO sortdata.SortParameters: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] Number of intermediate file to be merged: 20
> 17/05/15 18:07:08 INFO sortdata.SortParameters: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] File Buffer Size: 1048576
> 17/05/15 18:07:08 INFO sortdata.SortParameters: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] temp file location/tmp/26818412296318/0/vinod/sorttable4_offheap_inmemory/Fact/Part0/Segment_0/0/sortrowtmp
> 17/05/15 18:07:08 INFO newflow.DataLoadExecutor: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] Data Loading is started for table sorttable4_offheap_inmemory
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-17 14:32:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-12 07:11:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-09 01:15:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-05 10:36:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-06 01:35:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-23 15:29:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-16 21:59:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-21 10:00:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-23 04:03:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-25 02:54:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-22 13:48:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-07 07:22:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-12 09:42:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-28 14:58:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-29 07:54:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-09 03:39:52 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-03 06:08:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-06 18:55:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-19 22:23:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-09 04:39:45 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-22 02:16:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-22 15:45:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-18 07:47:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-10 08:09:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-19 09:00:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-08 00:01:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-23 06:39:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-15 08:25:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-05 16:23:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-25 04:01:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AZ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-29 10:59:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-14 18:03:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-22 07:22:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-17 09:19:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-10 23:47:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-25 03:00:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-06 07:24:23 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-19 21:27:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-19 04:38:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-28 11:34:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-10 04:22:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-26 10:10:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-31 19:37:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-08 16:39:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-11 12:56:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-17 16:09:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-18 02:29:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-25 11:04:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-15 06:46:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-27 19:40:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-28 05:06:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-19 01:57:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-31 08:17:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-30 21:12:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : VA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-11 08:33:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-01 17:48:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-26 23:34:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-29 09:13:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-15 12:32:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-28 23:07:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-01 14:19:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-28 10:02:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-30 23:28:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-19 22:35:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : GA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-21 04:27:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-24 18:08:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-15 05:12:14 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-09 13:06:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-24 03:19:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-19 14:30:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-28 02:25:18 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-09 19:01:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-07 20:59:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-04 14:01:14 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ME   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-30 04:57:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-07 16:45:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : GA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-28 22:23:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-06 23:04:16 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-17 04:00:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-12 17:43:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-15 09:16:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-11 08:17:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-22 00:55:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-03 22:22:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-09 01:50:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-10 17:03:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-21 18:38:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-28 16:14:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-30 06:03:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-02 22:37:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-25 19:39:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-06 07:47:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-28 02:20:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-25 16:52:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-19 01:07:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-13 05:56:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-10 16:38:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-17 13:30:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : RI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-05 03:41:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-24 11:28:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-31 02:06:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-25 14:27:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-04 19:37:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-21 10:56:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-17 08:42:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-20 21:09:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-01 22:21:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-12 13:38:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-07 17:17:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-05 06:07:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-02 06:14:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-19 04:38:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : VA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-05 20:03:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-06 06:05:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-02 03:16:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-14 09:46:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-14 16:56:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-01 01:25:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-15 22:32:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-02 17:32:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-02 06:35:18 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-01 20:42:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-17 18:57:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-17 04:36:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-09 12:21:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-23 16:19:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-27 08:11:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-26 04:15:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-07 13:40:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-22 10:05:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-06 11:49:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-02 19:32:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-22 17:48:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-27 13:48:52 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-15 14:52:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-10 01:38:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-21 03:09:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-21 05:54:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-13 11:32:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-07 07:27:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MS   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-02 05:34:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-28 07:59:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-19 15:34:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-28 14:53:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-17 09:44:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-24 18:48:47 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-05 22:55:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-24 03:11:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-05 13:19:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-15 04:53:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-17 05:41:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-19 15:43:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-11 23:55:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-03 11:57:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-01 01:31:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-29 23:15:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-07 21:45:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-24 11:22:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-19 09:31:16 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-11 02:11:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MS   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-14 08:40:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-16 03:57:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MD   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-05 09:59:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-05 07:36:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : VA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-25 16:45:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-23 13:32:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-07 10:03:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-28 09:03:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : SC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-28 16:46:14 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-03 10:13:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-25 10:14:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-09 20:38:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-21 19:51:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-11 21:29:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-14 18:07:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-19 17:19:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : UT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-31 07:10:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-02 18:26:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AZ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-17 02:57:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-17 14:51:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-14 15:41:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-15 11:12:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-26 19:27:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-02 18:08:45 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-11 20:31:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-01 21:27:23 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-13 14:23:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-23 17:31:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-25 20:24:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-17 12:40:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-31 21:49:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-14 08:07:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : VA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-12 17:23:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-06 11:53:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-08 04:30:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-02 10:26:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : GA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-09 00:21:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-05 16:37:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-29 11:46:52 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-08 09:09:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-30 19:00:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-17 06:19:45 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-03 20:08:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-28 05:35:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-08 17:41:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-01 08:37:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-19 19:21:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-26 22:21:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-17 21:47:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-10 19:07:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-20 21:45:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-13 06:10:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-01 23:30:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-10 16:58:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-04 06:07:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-06 06:22:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-05 10:37:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-20 10:51:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-19 05:23:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-26 06:09:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-10 05:46:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-14 18:53:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-01 20:19:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-08 19:20:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-10 13:32:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-03 20:51:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-07 00:45:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-15 02:48:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-07 18:51:47 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-21 07:32:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-03 18:11:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-09 14:24:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MD   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-11 18:43:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-17 04:05:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-16 19:14:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-01 21:10:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-12 08:22:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-29 02:28:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-04 23:13:11 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-14 20:38:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-29 02:15:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-26 11:57:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-16 23:44:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-03 18:16:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-06 19:57:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-09 20:22:18 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-08 15:19:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-26 06:48:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-05 20:30:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-14 12:33:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-27 07:28:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-26 19:02:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-05 11:48:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-01 20:51:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-28 23:03:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-26 22:17:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ND   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-16 12:38:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-11 09:57:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-23 13:52:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-14 00:39:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-21 12:48:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-18 02:00:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-14 17:31:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-04 10:51:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ND   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-01 17:30:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-30 01:57:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-18 07:42:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-18 06:58:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-30 23:42:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-27 17:22:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-18 14:30:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-27 05:12:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-10 00:12:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-16 19:29:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-19 00:15:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-14 01:50:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-16 07:22:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-12 15:21:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-30 08:06:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-29 07:48:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-22 07:46:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-25 02:37:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-01 20:05:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-18 23:50:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : VT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-22 06:47:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-19 12:14:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-02 05:51:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-10 01:13:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-24 05:47:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-18 21:19:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-13 16:28:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-15 15:24:23 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-17 22:45:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-27 02:28:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-28 01:47:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-19 09:10:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-03 12:48:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-14 13:32:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-28 12:47:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-30 05:04:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-03 08:21:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-23 17:35:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ME   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-04 07:09:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-22 19:21:18 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-27 08:12:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-25 13:50:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-17 23:54:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-12 11:29:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-15 07:36:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-14 19:33:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-09 20:44:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-14 02:19:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-05 18:53:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-29 11:42:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-01 22:22:11 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-27 03:50:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-24 00:09:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-05 07:09:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-21 09:49:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-03 00:49:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ND   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-06 21:14:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-15 05:38:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-17 06:56:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-29 18:37:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-26 21:50:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-04 17:56:45 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-30 01:27:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-12 22:22:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-20 23:39:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-28 18:46:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-04 09:46:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-27 06:35:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-01 16:15:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-25 23:45:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-16 18:54:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-04 09:01:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-06 06:05:47 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-16 08:09:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-09 07:12:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-09 16:20:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-20 17:41:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-13 08:51:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : DC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-06 08:07:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-16 23:50:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-02 22:17:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-20 09:40:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-05 06:55:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-27 22:16:14 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : UT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-05 15:06:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-25 06:24:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-24 15:41:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-14 17:08:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-30 21:43:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-05 13:49:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-18 11:09:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-20 09:24:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-01 00:38:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-27 15:43:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-23 14:39:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-02 21:33:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-15 06:26:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-25 11:03:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-20 21:37:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-30 15:51:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-31 07:42:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-04 05:53:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ME   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-17 14:18:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-13 03:29:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-24 18:13:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-29 20:47:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-21 17:15:47 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-22 13:03:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ME   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-04 15:10:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-14 11:17:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-14 19:32:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-22 22:49:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-01 11:53:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-06 20:41:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-17 15:08:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-25 16:41:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-04 21:53:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-13 09:32:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-08 02:46:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-28 22:53:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-16 20:52:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-20 10:33:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-02 06:22:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-13 06:55:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-10 15:58:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-21 09:27:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-27 04:12:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-04 00:29:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-30 21:17:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-30 12:30:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Market Planning Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-20 09:09:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-22 07:39:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-27 12:34:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-04 10:33:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Analyst to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-30 08:36:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-30 09:57:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Senior Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-23 08:10:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-03 12:56:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KS   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-15 10:44:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-14 23:57:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-18 08:43:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-17 15:00:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-16 23:36:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-15 08:06:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-25 16:40:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-01 14:12:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-25 06:49:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-28 04:06:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-03 06:59:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-26 09:06:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : LA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-21 20:13:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-19 19:05:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : GA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-11 22:19:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-20 16:29:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-02 13:10:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-06 11:45:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ND   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-05 20:16:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-19 12:08:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-11 01:10:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-11 07:28:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-17 20:52:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-19 03:30:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-19 00:30:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-29 08:21:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-06 10:00:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-15 09:51:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-10 12:29:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-10 05:48:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-17 09:33:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-06 07:20:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-30 05:57:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-20 08:21:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-12 23:44:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-03 13:16:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-28 17:33:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-04 10:41:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-05 01:40:47 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-19 04:24:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-11 02:09:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-06 13:00:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-07 00:30:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-29 04:30:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-29 12:12:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-26 22:35:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : GA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-19 19:03:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-12 06:52:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-06 20:31:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-26 22:59:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-17 11:43:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-11 14:29:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-02 23:54:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-01 11:18:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Consultant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-12 15:58:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-14 06:04:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Information Delivery to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-16 19:09:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-25 20:50:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Location Scout to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-21 15:24:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-12 09:18:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-02 19:18:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-17 20:49:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Report and Visualization Developer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-07 18:04:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-14 13:46:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-18 03:24:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-19 04:50:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-07 17:17:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-30 17:52:30 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-13 17:24:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-30 17:43:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KS   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-17 05:10:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-14 03:50:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-24 19:10:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-06 18:26:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-29 15:08:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-22 06:02:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ND   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-24 12:34:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-17 23:32:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-05 16:06:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-11 11:34:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-18 16:50:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-08 16:32:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : LA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-06 18:52:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-28 22:14:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KS   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-25 21:03:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-20 08:40:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-22 11:42:45 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-19 05:48:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-17 17:10:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-30 17:38:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-09 23:33:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-11 03:40:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-14 06:41:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-11 23:31:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-24 03:28:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-28 19:51:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-10 17:25:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-20 18:09:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-12 19:51:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-22 12:05:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-18 08:35:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-28 20:35:23 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : RI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-09 20:49:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-30 12:30:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-13 12:15:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-14 20:33:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-13 04:47:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-07 03:12:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-13 11:11:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-09 23:07:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-03 01:13:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-19 12:24:47 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-13 14:40:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-08 07:50:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : First Officer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-16 09:44:11 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-01 22:31:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ND   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Leader to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-11 20:18:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-06 13:17:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-30 04:31:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-26 00:14:14 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Research Associate to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-13 05:05:25 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-23 00:15:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-01 16:13:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-11 00:13:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-26 21:29:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-28 12:49:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 INFO storage.BlockManagerInfo: Removed broadcast_6_piece0 on 192.168.2.179:46491 in memory (size: 10.0 KB, free: 2.5 GB)
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-21 11:42:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-09 16:12:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ND   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-26 14:48:14 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-28 03:11:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-07 02:02:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-23 21:41:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-02 18:03:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-21 08:15:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-04 22:49:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-25 05:41:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-24 11:41:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-14 00:32:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-11 08:14:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-28 23:40:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-03 17:16:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-07 04:51:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-11 04:54:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-21 08:45:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MD   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-07 17:33:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-01 12:08:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-01 13:22:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-26 13:30:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-24 12:02:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-17 14:09:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-08 06:31:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-28 11:54:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WV   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-15 15:43:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-25 18:52:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-14 21:34:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-13 13:49:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : VA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-21 14:49:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-24 10:09:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Strategy Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-10 11:15:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-03 07:19:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-23 11:07:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-03 17:34:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-22 04:52:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-01 22:06:52 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : LA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-20 04:18:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-27 17:11:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AZ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-10 18:24:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-31 09:24:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-08 12:06:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-30 11:37:47 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-26 08:59:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-14 23:28:52 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : VT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-01 12:00:24 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-16 05:46:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-29 17:31:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-27 20:28:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-07 03:43:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-04 20:28:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : GA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-30 22:16:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-24 09:01:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-06 06:36:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-03 22:38:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 INFO storage.BlockManagerInfo: Removed broadcast_7_piece0 on 192.168.2.179:46491 in memory (size: 3.7 KB, free: 2.5 GB)
> 17/05/15 18:07:08 INFO storage.BlockManagerInfo: Removed broadcast_5_piece0 on 192.168.2.179:46491 in memory (size: 23.4 KB, free: 2.5 GB)
> 17/05/15 18:07:08 INFO spark.ContextCleaner: Cleaned accumulator 218
> 17/05/15 18:07:08 INFO spark.ContextCleaner: Cleaned accumulator 219
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-30 23:34:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-29 22:23:23 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-23 13:10:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-19 04:35:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-13 06:14:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-09 11:35:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-25 23:53:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-12 16:34:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 INFO spark.ContextCleaner: Cleaned shuffle 1
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-30 04:11:42 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-05 14:15:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 INFO storage.BlockManagerInfo: Removed broadcast_3_piece0 on 192.168.2.179:46491 in memory (size: 23.1 KB, free: 2.5 GB)
> 17/05/15 18:07:08 INFO storage.BlockManagerInfo: Removed broadcast_2_piece0 on 192.168.2.179:46491 in memory (size: 3.7 KB, free: 2.5 GB)
> 17/05/15 18:07:08 INFO storage.BlockManagerInfo: Removed broadcast_1_piece0 on 192.168.2.179:46491 in memory (size: 10.1 KB, free: 2.5 GB)
> 17/05/15 18:07:08 INFO spark.ContextCleaner: Cleaned shuffle 0
> 17/05/15 18:07:08 INFO spark.ContextCleaner: Cleaned accumulator 1
> 17/05/15 18:07:08 INFO spark.ContextCleaner: Cleaned accumulator 0
> 17/05/15 18:07:08 INFO storage.BlockManagerInfo: Removed broadcast_0_piece0 on 192.168.2.179:46491 in memory (size: 23.4 KB, free: 2.5 GB)
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-03 17:24:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-17 02:49:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-08 06:33:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-23 20:20:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-20 08:05:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-06 22:48:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-09 23:46:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-30 11:11:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : UT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-06 11:23:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-05 19:38:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-17 16:29:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-24 10:13:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-10 15:46:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-09 15:30:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-17 20:59:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-11 02:50:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-29 08:41:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-29 12:24:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-21 19:01:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-30 05:51:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AZ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-16 20:44:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-29 11:37:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-23 10:32:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-24 07:43:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-09 09:15:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-09 00:21:23 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-24 10:34:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-18 10:36:28 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-06 21:57:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-19 09:51:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-26 06:44:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-03 05:48:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-19 18:14:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-04 21:40:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-17 15:20:46 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-16 12:40:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-24 10:11:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-20 00:39:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-05 12:44:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-03 07:58:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-19 19:19:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-22 09:16:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-09 04:03:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-30 09:38:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-30 03:27:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-05 00:21:31 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-23 02:17:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-09 10:16:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-26 11:22:49 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-27 01:24:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Director to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-12 19:36:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-09 08:04:19 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ND   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Human Resources Partner to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-08 04:31:11 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-03 01:23:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Production Assistant to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-09 14:22:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-23 14:22:53 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AZ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Safety Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-10 16:37:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-12 21:51:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MO   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-10 12:17:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-30 06:41:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-04 12:40:18 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-09 08:15:16 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-31 09:01:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-20 02:22:32 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-15 05:32:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-22 23:25:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-07 18:31:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-28 13:41:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : WA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-14 15:30:38 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-11 14:45:29 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-21 15:08:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-03 01:28:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TN   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-16 04:59:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-03 00:57:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-16 06:30:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-19 15:01:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-11 21:52:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-09 14:15:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-17 14:30:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-07 05:38:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : VA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-25 09:56:11 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-22 08:00:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : RI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-04 07:35:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-18 20:16:14 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-12 00:59:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-15 00:45:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-18 12:50:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-28 12:12:18 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-10 14:50:43 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-17 19:01:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : SC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-30 01:47:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-24 07:00:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-31 15:38:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-10 12:36:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-15 16:57:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-15 05:27:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Cabin Stewardess to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-12 09:50:17 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-07 22:00:27 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Group Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-26 00:00:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-17 05:12:18 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MI   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-15 06:47:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-07-28 04:25:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : OH   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-06 13:55:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-17 11:26:47 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-31 11:42:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-29 23:53:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-07 11:24:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-25 21:15:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-02 03:19:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-17 01:51:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-12 22:55:18 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-20 09:49:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-18 11:31:39 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-10 06:38:16 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-21 19:06:22 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-08 13:54:50 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : MA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-28 22:12:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-05 21:45:33 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-11 08:07:00 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-18 01:50:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ME   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-22 10:50:36 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-28 03:23:54 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-03 10:22:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-06 21:32:20 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-09 03:52:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-27 02:27:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-01 06:01:09 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-11-19 22:28:47 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-12-28 02:16:57 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-15 01:39:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-19 19:09:26 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-03 14:22:07 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : SC   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-04-07 03:54:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-05-13 11:46:48 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-10 02:09:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-02 22:53:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-02-10 00:04:12 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-06 12:46:18 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-10 17:21:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-24 11:00:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Medical Science Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-01 05:43:52 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-25 00:25:05 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : AL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Project Support to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-23 02:56:56 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-21 01:51:34 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Associate Producer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-10-08 10:34:02 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-10-20 17:30:44 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : FL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Scientist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-12 13:14:03 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-13 18:30:40 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solution Architect to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-09-25 09:41:06 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-09 21:37:59 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : ND   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-06-26 09:22:13 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-06-26 21:05:35 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PA   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-07-22 14:54:41 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-03-11 02:16:51 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : TX   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Line Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-08-12 09:49:21 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-04-19 17:38:37 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : CT   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Program Coordinator to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-11-15 00:23:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-01-29 20:10:45 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : KY   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Technology Manager to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-06 19:20:55 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-02-22 17:55:08 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : IL   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Clinical Specialist to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-05-18 08:46:15 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-09-17 21:18:10 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : NJ   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Solutions Designer to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-01-17 15:33:58 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-12-30 00:29:04 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : PR   to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : Thought Leader Liaison to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2015-03-04 23:12:01 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 WARN impl.MeasureFieldConverterImpl: pool-33-thread-1 Cant not convert : 2016-08-11 17:17:52 to Numeric type value. Value considered as null.
> 17/05/15 18:07:08 INFO sortdata.SortDataRows: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] File based sorting will be used
> 17/05/15 18:07:08 INFO newflow.AbstractDataLoadProcessorStep: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] Total rows processed in step Data Writer: 0
> 17/05/15 18:07:08 INFO newflow.AbstractDataLoadProcessorStep: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] Total rows processed in step Sort Processor: 10000
> 17/05/15 18:07:08 INFO newflow.AbstractDataLoadProcessorStep: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] Total rows processed in step Data Converter: 10000
> 17/05/15 18:07:08 INFO newflow.AbstractDataLoadProcessorStep: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] Total rows processed in step Input Processor: 10000
> 17/05/15 18:07:08 INFO rdd.NewCarbonDataLoadRDD: DataLoad failure
> org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException: Problem while shutdown the server 
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:114)
> 	at org.apache.carbondata.processing.newflow.steps.SortProcessorStepImpl.execute(SortProcessorStepImpl.java:61)
> 	at org.apache.carbondata.processing.newflow.steps.DataWriterProcessorStepImpl.execute(DataWriterProcessorStepImpl.java:82)
> 	at org.apache.carbondata.processing.newflow.DataLoadExecutor.execute(DataLoadExecutor.java:48)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD$$anon$1.<init>(NewCarbonDataLoadRDD.scala:241)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD.compute(NewCarbonDataLoadRDD.scala:219)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:99)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to [B
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:49)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:24)
> 	at java.util.TimSort.binarySort(TimSort.java:296)
> 	at java.util.TimSort.sort(TimSort.java:239)
> 	at java.util.Arrays.sort(Arrays.java:1438)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows.startSorting(SortDataRows.java:197)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.processRowToNextStep(ParallelReadMergeSorterImpl.java:170)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:111)
> 	... 13 more
> 17/05/15 18:07:08 ERROR rdd.NewCarbonDataLoadRDD: [Executor task launch worker-7][partitionID:vinod_sorttable4_offheap_inmemory_2552c8a8-99df-4024-a1fd-fc0c86e06d67] 
> org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException: Problem while shutdown the server 
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:114)
> 	at org.apache.carbondata.processing.newflow.steps.SortProcessorStepImpl.execute(SortProcessorStepImpl.java:61)
> 	at org.apache.carbondata.processing.newflow.steps.DataWriterProcessorStepImpl.execute(DataWriterProcessorStepImpl.java:82)
> 	at org.apache.carbondata.processing.newflow.DataLoadExecutor.execute(DataLoadExecutor.java:48)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD$$anon$1.<init>(NewCarbonDataLoadRDD.scala:241)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD.compute(NewCarbonDataLoadRDD.scala:219)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:99)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to [B
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:49)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:24)
> 	at java.util.TimSort.binarySort(TimSort.java:296)
> 	at java.util.TimSort.sort(TimSort.java:239)
> 	at java.util.Arrays.sort(Arrays.java:1438)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows.startSorting(SortDataRows.java:197)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.processRowToNextStep(ParallelReadMergeSorterImpl.java:170)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:111)
> 	... 13 more
> 17/05/15 18:07:08 ERROR executor.Executor: Exception in task 0.0 in stage 5.0 (TID 11)
> org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException: Problem while shutdown the server 
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:114)
> 	at org.apache.carbondata.processing.newflow.steps.SortProcessorStepImpl.execute(SortProcessorStepImpl.java:61)
> 	at org.apache.carbondata.processing.newflow.steps.DataWriterProcessorStepImpl.execute(DataWriterProcessorStepImpl.java:82)
> 	at org.apache.carbondata.processing.newflow.DataLoadExecutor.execute(DataLoadExecutor.java:48)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD$$anon$1.<init>(NewCarbonDataLoadRDD.scala:241)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD.compute(NewCarbonDataLoadRDD.scala:219)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:99)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to [B
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:49)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:24)
> 	at java.util.TimSort.binarySort(TimSort.java:296)
> 	at java.util.TimSort.sort(TimSort.java:239)
> 	at java.util.Arrays.sort(Arrays.java:1438)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows.startSorting(SortDataRows.java:197)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.processRowToNextStep(ParallelReadMergeSorterImpl.java:170)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:111)
> 	... 13 more
> 17/05/15 18:07:08 WARN scheduler.TaskSetManager: Lost task 0.0 in stage 5.0 (TID 11, localhost, executor driver): org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException: Problem while shutdown the server 
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:114)
> 	at org.apache.carbondata.processing.newflow.steps.SortProcessorStepImpl.execute(SortProcessorStepImpl.java:61)
> 	at org.apache.carbondata.processing.newflow.steps.DataWriterProcessorStepImpl.execute(DataWriterProcessorStepImpl.java:82)
> 	at org.apache.carbondata.processing.newflow.DataLoadExecutor.execute(DataLoadExecutor.java:48)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD$$anon$1.<init>(NewCarbonDataLoadRDD.scala:241)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD.compute(NewCarbonDataLoadRDD.scala:219)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:99)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to [B
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:49)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:24)
> 	at java.util.TimSort.binarySort(TimSort.java:296)
> 	at java.util.TimSort.sort(TimSort.java:239)
> 	at java.util.Arrays.sort(Arrays.java:1438)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows.startSorting(SortDataRows.java:197)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.processRowToNextStep(ParallelReadMergeSorterImpl.java:170)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:111)
> 	... 13 more
> 17/05/15 18:07:08 ERROR scheduler.TaskSetManager: Task 0 in stage 5.0 failed 1 times; aborting job
> 17/05/15 18:07:08 INFO scheduler.TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool 
> 17/05/15 18:07:08 INFO scheduler.TaskSchedulerImpl: Cancelling stage 5
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: ResultStage 5 (collect at CarbonDataRDDFactory.scala:630) failed in 0.659 s due to Job aborted due to stage failure: Task 0 in stage 5.0 failed 1 times, most recent failure: Lost task 0.0 in stage 5.0 (TID 11, localhost, executor driver): org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException: Problem while shutdown the server 
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:114)
> 	at org.apache.carbondata.processing.newflow.steps.SortProcessorStepImpl.execute(SortProcessorStepImpl.java:61)
> 	at org.apache.carbondata.processing.newflow.steps.DataWriterProcessorStepImpl.execute(DataWriterProcessorStepImpl.java:82)
> 	at org.apache.carbondata.processing.newflow.DataLoadExecutor.execute(DataLoadExecutor.java:48)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD$$anon$1.<init>(NewCarbonDataLoadRDD.scala:241)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD.compute(NewCarbonDataLoadRDD.scala:219)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:99)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to [B
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:49)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:24)
> 	at java.util.TimSort.binarySort(TimSort.java:296)
> 	at java.util.TimSort.sort(TimSort.java:239)
> 	at java.util.Arrays.sort(Arrays.java:1438)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows.startSorting(SortDataRows.java:197)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.processRowToNextStep(ParallelReadMergeSorterImpl.java:170)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:111)
> 	... 13 more
> Driver stacktrace:
> 17/05/15 18:07:08 INFO scheduler.DAGScheduler: Job 3 failed: collect at CarbonDataRDDFactory.scala:630, took 0.668890 s
> 17/05/15 18:07:08 INFO rdd.CarbonDataRDDFactory$: pool-24-thread-10 DataLoad failure: Problem while shutdown the server 
> 17/05/15 18:07:08 ERROR rdd.CarbonDataRDDFactory$: pool-24-thread-10 
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 5.0 failed 1 times, most recent failure: Lost task 0.0 in stage 5.0 (TID 11, localhost, executor driver): org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException: Problem while shutdown the server 
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:114)
> 	at org.apache.carbondata.processing.newflow.steps.SortProcessorStepImpl.execute(SortProcessorStepImpl.java:61)
> 	at org.apache.carbondata.processing.newflow.steps.DataWriterProcessorStepImpl.execute(DataWriterProcessorStepImpl.java:82)
> 	at org.apache.carbondata.processing.newflow.DataLoadExecutor.execute(DataLoadExecutor.java:48)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD$$anon$1.<init>(NewCarbonDataLoadRDD.scala:241)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD.compute(NewCarbonDataLoadRDD.scala:219)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:99)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to [B
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:49)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:24)
> 	at java.util.TimSort.binarySort(TimSort.java:296)
> 	at java.util.TimSort.sort(TimSort.java:239)
> 	at java.util.Arrays.sort(Arrays.java:1438)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows.startSorting(SortDataRows.java:197)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.processRowToNextStep(ParallelReadMergeSorterImpl.java:170)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:111)
> 	... 13 more
> Driver stacktrace:
> 	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1435)
> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1423)
> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1422)
> 	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> 	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> 	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1422)
> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
> 	at scala.Option.foreach(Option.scala:257)
> 	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:802)
> 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1650)
> 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1605)
> 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1594)
> 	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
> 	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:628)
> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1918)
> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1931)
> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1944)
> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1958)
> 	at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:935)
> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
> 	at org.apache.spark.rdd.RDD.withScope(RDD.scala:362)
> 	at org.apache.spark.rdd.RDD.collect(RDD.scala:934)
> 	at org.apache.carbondata.spark.rdd.CarbonDataRDDFactory$.loadDataFile$1(CarbonDataRDDFactory.scala:630)
> 	at org.apache.carbondata.spark.rdd.CarbonDataRDDFactory$.loadCarbonData(CarbonDataRDDFactory.scala:691)
> 	at org.apache.spark.sql.execution.command.LoadTable.run(carbonTableSchema.scala:560)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
> 	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
> 	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
> 	at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:87)
> 	at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:87)
> 	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:185)
> 	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:64)
> 	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:592)
> 	at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:699)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:220)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:163)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:160)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:422)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:173)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException: Problem while shutdown the server 
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:114)
> 	at org.apache.carbondata.processing.newflow.steps.SortProcessorStepImpl.execute(SortProcessorStepImpl.java:61)
> 	at org.apache.carbondata.processing.newflow.steps.DataWriterProcessorStepImpl.execute(DataWriterProcessorStepImpl.java:82)
> 	at org.apache.carbondata.processing.newflow.DataLoadExecutor.execute(DataLoadExecutor.java:48)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD$$anon$1.<init>(NewCarbonDataLoadRDD.scala:241)
> 	at org.apache.carbondata.spark.rdd.NewCarbonDataLoadRDD.compute(NewCarbonDataLoadRDD.scala:219)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:99)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
> 	... 3 more
> Caused by: java.lang.ClassCastException: java.lang.String cannot be cast to [B
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:49)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator.compare(NewRowComparator.java:24)
> 	at java.util.TimSort.binarySort(TimSort.java:296)
> 	at java.util.TimSort.sort(TimSort.java:239)
> 	at java.util.Arrays.sort(Arrays.java:1438)
> 	at org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows.startSorting(SortDataRows.java:197)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.processRowToNextStep(ParallelReadMergeSorterImpl.java:170)
> 	at org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl.sort(ParallelReadMergeSorterImpl.java:111)
> 	... 13 more
> 17/05/15 18:07:08 INFO rdd.CarbonDataRDDFactory$: pool-24-thread-10 ********starting clean up**********
> 17/05/15 18:07:08 INFO load.CarbonLoaderUtil: pool-34-thread-1 Deleted the local store location/tmp/26818412296318/0 : TIme taken: 0
> 17/05/15 18:07:08 INFO rdd.CarbonDataRDDFactory$: pool-24-thread-10 ********clean up done**********
> 17/05/15 18:07:08 AUDIT rdd.CarbonDataRDDFactory$: [vinod-Vostro-3559][anonymous][Thread-179]Data load is failed for vinod.sorttable4_offheap_inmemory
> 17/05/15 18:07:08 WARN rdd.CarbonDataRDDFactory$: pool-24-thread-10 Cannot write load metadata file as data load failed
> 17/05/15 18:07:08 ERROR command.LoadTable: pool-24-thread-10 
> java.lang.Exception: DataLoad failure: Problem while shutdown the server 
> 	at org.apache.carbondata.spark.rdd.CarbonDataRDDFactory$.loadCarbonData(CarbonDataRDDFactory.scala:834)
> 	at org.apache.spark.sql.execution.command.LoadTable.run(carbonTableSchema.scala:560)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
> 	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
> 	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
> 	at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:87)
> 	at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:87)
> 	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:185)
> 	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:64)
> 	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:592)
> 	at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:699)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:220)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:163)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:160)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:422)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:173)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> 17/05/15 18:07:08 AUDIT command.LoadTable: [vinod-Vostro-3559][anonymous][Thread-179]Dataload failure for vinod.sorttable4_offheap_inmemory. Please check the logs
> 17/05/15 18:07:08 INFO locks.HdfsFileLock: pool-24-thread-10 Deleted the lock file hdfs://localhost:54310/opt/carbonStore/vinod/sorttable4_offheap_inmemory/meta.lock
> 17/05/15 18:07:08 INFO command.LoadTable: pool-24-thread-10 Table MetaData Unlocked Successfully after data load
> 17/05/15 18:07:08 ERROR thriftserver.SparkExecuteStatementOperation: Error executing query, currentState RUNNING, 
> java.lang.Exception: DataLoad failure: Problem while shutdown the server 
> 	at org.apache.carbondata.spark.rdd.CarbonDataRDDFactory$.loadCarbonData(CarbonDataRDDFactory.scala:834)
> 	at org.apache.spark.sql.execution.command.LoadTable.run(carbonTableSchema.scala:560)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
> 	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
> 	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
> 	at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:87)
> 	at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:87)
> 	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:185)
> 	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:64)
> 	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:592)
> 	at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:699)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:220)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:163)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:160)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:422)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:173)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> 17/05/15 18:07:08 ERROR thriftserver.SparkExecuteStatementOperation: Error running hive query: 
> org.apache.hive.service.cli.HiveSQLException: java.lang.Exception: DataLoad failure: Problem while shutdown the server 
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:258)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:163)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:160)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:422)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:173)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)